You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2015/09/16 21:18:19 UTC

[01/50] [abbrv] hive git commit: HIVE-11329: Column prefix in key of hbase column prefix map (Wojciech Indyk, reviewed by Sergio Pena)

Repository: hive
Updated Branches:
  refs/heads/spark f78f66359 -> 70eeadd2f


HIVE-11329: Column prefix in key of hbase column prefix map (Wojciech Indyk, reviewed by Sergio Pena)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d51c62a4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d51c62a4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d51c62a4

Branch: refs/heads/spark
Commit: d51c62a455eb08ee49f10ea2e117ca90de0bf47b
Parents: 7281a46
Author: Sergio Pena <se...@cloudera.com>
Authored: Tue Sep 8 08:39:10 2015 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Tue Sep 8 08:39:10 2015 -0500

----------------------------------------------------------------------
 .../hadoop/hive/hbase/ColumnMappings.java       |  5 ++
 .../apache/hadoop/hive/hbase/HBaseSerDe.java    | 19 +++++-
 .../hadoop/hive/hbase/HBaseSerDeParameters.java |  8 ++-
 .../hadoop/hive/hbase/HBaseStorageHandler.java  | 13 ++--
 .../hadoop/hive/hbase/LazyHBaseCellMap.java     | 19 +++++-
 .../apache/hadoop/hive/hbase/LazyHBaseRow.java  |  5 +-
 .../hadoop/hive/hbase/LazyHBaseCellMapTest.java | 72 ++++++++++++++++++++
 .../positive/hbase_binary_map_queries_prefix.q  | 15 +++-
 .../hbase_binary_map_queries_prefix.q.out       | 40 +++++++++++
 9 files changed, 177 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
index 5cb3752..d7d4461 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
@@ -354,6 +354,7 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
     String mappingSpec;
     String qualifierPrefix;
     byte[] qualifierPrefixBytes;
+    boolean doPrefixCut;
 
     public String getColumnName() {
       return columnName;
@@ -399,6 +400,10 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
       return qualifierPrefixBytes;
     }
 
+    public boolean isDoPrefixCut(){
+      return doPrefixCut;
+    }
+
     public boolean isCategory(ObjectInspector.Category category) {
       return columnType.getCategory() == category;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java
index e843bd8..ed2df5f 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java
@@ -93,6 +93,13 @@ public class HBaseSerDe extends AbstractSerDe {
    **/
   public static final String SERIALIZATION_TYPE = "serialization.type";
 
+  /**
+   * Defines if the prefix column from hbase should be hidden.
+   * It works only when @HBASE_COLUMNS_REGEX_MATCHING is true.
+   * Default value of this parameter is false
+   */
+  public static final String HBASE_COLUMNS_PREFIX_HIDE = "hbase.columns.mapping.prefix.hide";
+
   private ObjectInspector cachedObjectInspector;
   private LazyHBaseRow cachedHBaseRow;
 
@@ -136,6 +143,11 @@ public class HBaseSerDe extends AbstractSerDe {
       throws SerDeException {
     return parseColumnsMapping(columnsMappingSpec, true);
   }
+
+  public static ColumnMappings parseColumnsMapping(
+          String columnsMappingSpec, boolean doColumnRegexMatching) throws SerDeException {
+	return parseColumnsMapping(columnsMappingSpec, doColumnRegexMatching, false);
+  }
   /**
    * Parses the HBase columns mapping specifier to identify the column families, qualifiers
    * and also caches the byte arrays corresponding to them. One of the Hive table
@@ -143,11 +155,12 @@ public class HBaseSerDe extends AbstractSerDe {
    *
    * @param columnsMappingSpec string hbase.columns.mapping specified when creating table
    * @param doColumnRegexMatching whether to do a regex matching on the columns or not
+   * @param hideColumnPrefix whether to hide a prefix of column mapping in key name in a map (works only if @doColumnRegexMatching is true)
    * @return List<ColumnMapping> which contains the column mapping information by position
    * @throws org.apache.hadoop.hive.serde2.SerDeException
    */
   public static ColumnMappings parseColumnsMapping(
-      String columnsMappingSpec, boolean doColumnRegexMatching) throws SerDeException {
+      String columnsMappingSpec, boolean doColumnRegexMatching, boolean hideColumnPrefix) throws SerDeException {
 
     if (columnsMappingSpec == null) {
       throw new SerDeException("Error: hbase.columns.mapping missing for this HBase table.");
@@ -206,6 +219,8 @@ public class HBaseSerDe extends AbstractSerDe {
             // we have a prefix with a wildcard
             columnMapping.qualifierPrefix = parts[1].substring(0, parts[1].length() - 2);
             columnMapping.qualifierPrefixBytes = Bytes.toBytes(columnMapping.qualifierPrefix);
+            //pass a flag to hide prefixes
+            columnMapping.doPrefixCut=hideColumnPrefix;
             // we weren't provided any actual qualifier name. Set these to
             // null.
             columnMapping.qualifierName = null;
@@ -214,6 +229,8 @@ public class HBaseSerDe extends AbstractSerDe {
             // set the regular provided qualifier names
             columnMapping.qualifierName = parts[1];
             columnMapping.qualifierNameBytes = Bytes.toBytes(parts[1]);
+            //if there is no prefix then we don't cut anything
+            columnMapping.doPrefixCut=false;
           }
         } else {
           columnMapping.qualifierName = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDeParameters.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDeParameters.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDeParameters.java
index 71f5da5..43c1f0c 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDeParameters.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDeParameters.java
@@ -22,6 +22,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
+import javax.annotation.Nullable;
+
 import org.apache.avro.Schema;
 import org.apache.avro.reflect.ReflectData;
 import org.apache.hadoop.conf.Configuration;
@@ -38,8 +40,6 @@ import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.util.ReflectionUtils;
 
-import javax.annotation.Nullable;
-
 /**
  * HBaseSerDeParameters encapsulates SerDeParameters and additional configurations that are specific for
  * HBaseSerDe.
@@ -57,6 +57,7 @@ public class HBaseSerDeParameters {
   private final String columnMappingString;
   private final ColumnMappings columnMappings;
   private final boolean doColumnRegexMatching;
+  private final boolean doColumnPrefixCut;
 
   private final long putTimestamp;
   private final HBaseKeyFactory keyFactory;
@@ -69,8 +70,9 @@ public class HBaseSerDeParameters {
     columnMappingString = tbl.getProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING);
     doColumnRegexMatching =
         Boolean.valueOf(tbl.getProperty(HBaseSerDe.HBASE_COLUMNS_REGEX_MATCHING, "true"));
+    doColumnPrefixCut = Boolean.valueOf(tbl.getProperty(HBaseSerDe.HBASE_COLUMNS_PREFIX_HIDE, "false"));
     // Parse and initialize the HBase columns mapping
-    columnMappings = HBaseSerDe.parseColumnsMapping(columnMappingString, doColumnRegexMatching);
+    columnMappings = HBaseSerDe.parseColumnsMapping(columnMappingString, doColumnRegexMatching, doColumnPrefixCut);
 
     // Build the type property string if not supplied
     String columnTypeProperty = tbl.getProperty(serdeConstants.LIST_COLUMN_TYPES);

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
index c0eff23..cedb6e0 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
@@ -44,13 +44,10 @@ import org.apache.hadoop.hbase.mapred.TableOutputFormat;
 import org.apache.hadoop.hbase.mapreduce.TableInputFormatBase;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
-import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
+import org.apache.hadoop.hbase.security.token.TokenUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping;
-import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hive.metastore.HiveMetaHook;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -66,15 +63,11 @@ import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.SerDe;
 import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.hbase.security.token.TokenUtil;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.zookeeper.KeeperException;
 
 import com.yammer.metrics.core.MetricsRegistry;
 
@@ -348,7 +341,9 @@ public class HBaseStorageHandler extends DefaultStorageHandler
       HBaseSerDe.HBASE_COLUMNS_MAPPING,
       tableProperties.getProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING));
     jobProperties.put(HBaseSerDe.HBASE_COLUMNS_REGEX_MATCHING,
-        tableProperties.getProperty(HBaseSerDe.HBASE_COLUMNS_REGEX_MATCHING, "true"));
+            tableProperties.getProperty(HBaseSerDe.HBASE_COLUMNS_REGEX_MATCHING, "true"));
+    jobProperties.put(HBaseSerDe.HBASE_COLUMNS_PREFIX_HIDE,
+            tableProperties.getProperty(HBaseSerDe.HBASE_COLUMNS_PREFIX_HIDE, "false"));
     jobProperties.put(HBaseSerDe.HBASE_TABLE_DEFAULT_STORAGE_TYPE,
       tableProperties.getProperty(HBaseSerDe.HBASE_TABLE_DEFAULT_STORAGE_TYPE,"string"));
     String scanCache = tableProperties.getProperty(HBaseSerDe.HBASE_SCAN_CACHE);

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseCellMap.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseCellMap.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseCellMap.java
index 09cbf52..459d934 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseCellMap.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseCellMap.java
@@ -45,8 +45,9 @@ public class LazyHBaseCellMap extends LazyMap {
   private byte [] columnFamilyBytes;
   private byte[] qualPrefix;
   private List<Boolean> binaryStorage;
+  private boolean hideQualPrefix;
 
-  /**
+	/**
    * Construct a LazyCellMap object with the ObjectInspector.
    * @param oi
    */
@@ -62,15 +63,23 @@ public class LazyHBaseCellMap extends LazyMap {
     init(r, columnFamilyBytes, binaryStorage, null);
   }
 
+	public void init(
+			Result r,
+			byte [] columnFamilyBytes,
+			List<Boolean> binaryStorage, byte[] qualPrefix) {
+		init(r, columnFamilyBytes, binaryStorage, qualPrefix, false);
+	}
+
   public void init(
       Result r,
       byte [] columnFamilyBytes,
-      List<Boolean> binaryStorage, byte[] qualPrefix) {
+      List<Boolean> binaryStorage, byte[] qualPrefix, boolean hideQualPrefix) {
     this.isNull = false;
     this.result = r;
     this.columnFamilyBytes = columnFamilyBytes;
     this.binaryStorage = binaryStorage;
     this.qualPrefix = qualPrefix;
+    this.hideQualPrefix = hideQualPrefix;
     setParsed(false);
   }
 
@@ -106,7 +115,11 @@ public class LazyHBaseCellMap extends LazyMap {
               binaryStorage.get(0));
 
         ByteArrayRef keyRef = new ByteArrayRef();
-        keyRef.setData(e.getKey());
+		  if (qualPrefix!=null && hideQualPrefix){
+			  keyRef.setData(Bytes.tail(e.getKey(), e.getKey().length-qualPrefix.length)); //cut prefix from hive's map key
+		  }else{
+			  keyRef.setData(e.getKey()); //for non-prefix maps
+		  }
         key.init(keyRef, 0, keyRef.getData().length);
 
         // Value

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
index 868d81f..003dd1c 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping;
 import org.apache.hadoop.hive.hbase.struct.HBaseValueFactory;
@@ -36,6 +35,8 @@ import org.apache.hadoop.hive.serde2.lazy.LazyTimestamp;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazySimpleStructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * LazyObject for storing an HBase row.  The field of an HBase row can be
  * primitive or non-primitive.
@@ -148,7 +149,7 @@ public class LazyHBaseRow extends LazyStruct {
         // qualifier prefix to cherry pick the qualifiers that match the prefix instead of picking
         // up everything
         ((LazyHBaseCellMap) fields[fieldID]).init(
-            result, colMap.familyNameBytes, colMap.binaryStorage, colMap.qualifierPrefixBytes);
+            result, colMap.familyNameBytes, colMap.binaryStorage, colMap.qualifierPrefixBytes, colMap.isDoPrefixCut());
         return fields[fieldID].getObject();
       }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/test/org/apache/hadoop/hive/hbase/LazyHBaseCellMapTest.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/LazyHBaseCellMapTest.java b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/LazyHBaseCellMapTest.java
new file mode 100644
index 0000000..eb13e14
--- /dev/null
+++ b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/LazyHBaseCellMapTest.java
@@ -0,0 +1,72 @@
+/**
+ * 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.hbase;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hive.serde2.lazy.LazyFactory;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazyMapObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.Text;
+
+import junit.framework.TestCase;
+
+public class LazyHBaseCellMapTest extends TestCase {
+	public static final byte[] TEST_ROW = Bytes.toBytes("test-row");
+	public static final byte[] COLUMN_FAMILY = Bytes.toBytes("a");
+	public static final String QUAL_PREFIX = "col_";
+
+
+	public void testInitColumnPrefix() throws Exception {
+		Text nullSequence = new Text("\\N");
+		ObjectInspector oi = LazyFactory.createLazyObjectInspector(
+				TypeInfoUtils.getTypeInfosFromTypeString("map<string,string>").get(0),
+				new byte[] { (byte) 1, (byte) 2 }, 0, nullSequence, false, (byte) 0);
+
+		LazyHBaseCellMap b = new LazyHBaseCellMap((LazyMapObjectInspector) oi);
+
+		// Initialize a result
+		Cell[] cells = new KeyValue[2];
+
+		final String col1="1";
+		final String col2="2";
+		cells[0] = new KeyValue(TEST_ROW, COLUMN_FAMILY,
+				Bytes.toBytes(QUAL_PREFIX+col1), Bytes.toBytes("cfacol1"));
+		cells[1]=new KeyValue(TEST_ROW, COLUMN_FAMILY,
+				Bytes.toBytes(QUAL_PREFIX+col2), Bytes.toBytes("cfacol2"));
+
+		Result r = Result.create(cells);
+
+		List<Boolean> mapBinaryStorage = new ArrayList<Boolean>();
+		mapBinaryStorage.add(false);
+		mapBinaryStorage.add(false);
+
+		b.init(r, COLUMN_FAMILY, mapBinaryStorage, Bytes.toBytes(QUAL_PREFIX), true);
+
+		assertNotNull(b.getMapValueElement(new Text(col1)));
+		assertNotNull(b.getMapValueElement(new Text(col2)));
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q b/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q
index 3d3f5ea..9ff4366 100644
--- a/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q
+++ b/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q
@@ -49,4 +49,17 @@ TBLPROPERTIES ("hbase.table.name"="t_hive_maps");
 
 SELECT * FROM t_ext_hbase_maps ORDER BY key;
 
-DROP TABLE t_ext_hbase_maps;
\ No newline at end of file
+DROP TABLE t_ext_hbase_maps;
+
+DROP TABLE t_ext_hbase_maps_cut_prefix;
+
+CREATE EXTERNAL TABLE t_ext_hbase_maps_cut_prefix(key STRING,
+                                       string_map_cols MAP<STRING, STRING>, simple_string_col STRING)
+STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
+WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-string:string_.*,cf-string:simple_string_col"
+    ,"hbase.columns.mapping.prefix.hide"="true")
+TBLPROPERTIES ("hbase.table.name"="t_hive_maps");
+
+SELECT * FROM t_ext_hbase_maps_cut_prefix ORDER BY key;
+
+DROP TABLE t_ext_hbase_maps_cut_prefix;

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out b/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out
index 5d21134..f6432b3 100644
--- a/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out
@@ -152,3 +152,43 @@ POSTHOOK: query: DROP TABLE t_ext_hbase_maps
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@t_ext_hbase_maps
 POSTHOOK: Output: default@t_ext_hbase_maps
+PREHOOK: query: DROP TABLE t_ext_hbase_maps_cut_prefix
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE t_ext_hbase_maps_cut_prefix
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE EXTERNAL TABLE t_ext_hbase_maps_cut_prefix(key STRING,
+                                       string_map_cols MAP<STRING, STRING>, simple_string_col STRING)
+STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
+WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-string:string_.*,cf-string:simple_string_col"
+    ,"hbase.columns.mapping.prefix.hide"="true")
+TBLPROPERTIES ("hbase.table.name"="t_hive_maps")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t_ext_hbase_maps_cut_prefix
+POSTHOOK: query: CREATE EXTERNAL TABLE t_ext_hbase_maps_cut_prefix(key STRING,
+                                       string_map_cols MAP<STRING, STRING>, simple_string_col STRING)
+STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
+WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-string:string_.*,cf-string:simple_string_col"
+    ,"hbase.columns.mapping.prefix.hide"="true")
+TBLPROPERTIES ("hbase.table.name"="t_hive_maps")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t_ext_hbase_maps_cut_prefix
+PREHOOK: query: SELECT * FROM t_ext_hbase_maps_cut_prefix ORDER BY key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t_ext_hbase_maps_cut_prefix
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM t_ext_hbase_maps_cut_prefix ORDER BY key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t_ext_hbase_maps_cut_prefix
+#### A masked pattern was here ####
+125	{"col":"val_125"}	val_125
+126	{"col":"val_126"}	val_126
+PREHOOK: query: DROP TABLE t_ext_hbase_maps_cut_prefix
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t_ext_hbase_maps_cut_prefix
+PREHOOK: Output: default@t_ext_hbase_maps_cut_prefix
+POSTHOOK: query: DROP TABLE t_ext_hbase_maps_cut_prefix
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t_ext_hbase_maps_cut_prefix
+POSTHOOK: Output: default@t_ext_hbase_maps_cut_prefix


[15/50] [abbrv] hive git commit: HIVE-10980 : Merge of dynamic partitions loads all data to default partition (Illya Yalovyy via Ashutosh Chauhan)

Posted by xu...@apache.org.
HIVE-10980 : Merge of dynamic partitions loads all data to default partition (Illya Yalovyy via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ff1f5b1a
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ff1f5b1a
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ff1f5b1a

Branch: refs/heads/spark
Commit: ff1f5b1a7afc4c934bad2a39da217513760d0ba2
Parents: 1e97b16
Author: Illya Yalovyy <ya...@amazon.com>
Authored: Thu Sep 10 00:17:00 2015 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Sat Sep 12 19:50:08 2015 -0700

----------------------------------------------------------------------
 data/files/dynpartdata1.txt                     |  5 +
 data/files/dynpartdata2.txt                     |  6 ++
 .../hive/ql/optimizer/GenMapRedUtils.java       | 57 +++++++++--
 ...nMapRedUtilsUsePartitionColumnsNegative.java | 73 +++++++++++++++
 ...nMapRedUtilsUsePartitionColumnsPositive.java | 61 ++++++++++++
 .../test/queries/clientpositive/dynpart_merge.q | 28 ++++++
 .../results/clientpositive/dynpart_merge.q.out  | 99 ++++++++++++++++++++
 .../list_bucket_dml_6.q.java1.7.out             | 12 +--
 .../list_bucket_dml_6.q.java1.8.out             | 12 +--
 .../clientpositive/list_bucket_dml_7.q.out      | 12 +--
 10 files changed, 341 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ff1f5b1a/data/files/dynpartdata1.txt
----------------------------------------------------------------------
diff --git a/data/files/dynpartdata1.txt b/data/files/dynpartdata1.txt
new file mode 100644
index 0000000..aefb87f
--- /dev/null
+++ b/data/files/dynpartdata1.txt
@@ -0,0 +1,5 @@
+20150316,16,reqA,clusterIdA,cacheId1
+20150316,16,reqB,clusterIdB,cacheId2
+20150316,16,reqA,clusterIdC,cacheId3
+20150316,16,reqD,clusterIdD,cacheId4
+20150316,16,reqA,clusterIdA,cacheId5

http://git-wip-us.apache.org/repos/asf/hive/blob/ff1f5b1a/data/files/dynpartdata2.txt
----------------------------------------------------------------------
diff --git a/data/files/dynpartdata2.txt b/data/files/dynpartdata2.txt
new file mode 100644
index 0000000..4afdb7f
--- /dev/null
+++ b/data/files/dynpartdata2.txt
@@ -0,0 +1,6 @@
+20150317,16,reqB,clusterIdB,cacheId6
+20150318,16,reqA,clusterIdC,cacheId6
+20150317,15,reqD,clusterIdD,cacheId7
+20150316,16,reqA,clusterIdD,cacheId8
+20150316,16,reqD,clusterIdB,cacheId9
+20150316,16,reqB,clusterIdA,cacheId1

http://git-wip-us.apache.org/repos/asf/hive/blob/ff1f5b1a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
index 4a325fb..02fbdfe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.optimizer;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -112,6 +113,7 @@ import org.apache.hadoop.hive.ql.stats.StatsFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.mapred.InputFormat;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Interner;
 
 /**
@@ -1234,16 +1236,13 @@ public final class GenMapRedUtils {
       ArrayList<ColumnInfo> signature = inputRS.getSignature();
       String tblAlias = fsInputDesc.getTableInfo().getTableName();
       LinkedHashMap<String, String> colMap = new LinkedHashMap<String, String>();
-      StringBuilder partCols = new StringBuilder();
       for (String dpCol : dpCtx.getDPColNames()) {
         ColumnInfo colInfo = new ColumnInfo(dpCol,
             TypeInfoFactory.stringTypeInfo, // all partition column type should be string
             tblAlias, true); // partition column is virtual column
         signature.add(colInfo);
         colMap.put(dpCol, dpCol); // input and output have the same column name
-        partCols.append(dpCol).append('/');
       }
-      partCols.setLength(partCols.length() - 1); // remove the last '/'
       inputRS.setSignature(signature);
 
       // create another DynamicPartitionCtx, which has a different input-to-DP column mapping
@@ -1252,9 +1251,7 @@ public final class GenMapRedUtils {
       fsOutputDesc.setDynPartCtx(dpCtx2);
 
       // update the FileSinkOperator to include partition columns
-      fsInputDesc.getTableInfo().getProperties().setProperty(
-        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS,
-        partCols.toString()); // list of dynamic partition column names
+      usePartitionColumns(fsInputDesc.getTableInfo().getProperties(), dpCtx.getDPColNames());
     } else {
       // non-partitioned table
       fsInputDesc.getTableInfo().getProperties().remove(
@@ -1877,7 +1874,55 @@ public final class GenMapRedUtils {
     }
     return null;
   }
+  /**
+   * Uses only specified partition columns.
+   * Provided properties should be pre-populated with partition column names and types.
+   * This function retains only information related to the columns from the list.
+   * @param properties properties to update
+   * @param partColNames list of columns to use
+   */
+  static void usePartitionColumns(Properties properties, List<String> partColNames) {
+    Preconditions.checkArgument(!partColNames.isEmpty(), "No partition columns provided to use");
+    Preconditions.checkArgument(new HashSet<String>(partColNames).size() == partColNames.size(),
+        "Partition columns should be unique: " + partColNames);
+
+    String[] partNames = properties.getProperty(
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS)
+        .split("/");
+    String[] partTypes = properties.getProperty(
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES)
+        .split(":");
+    Preconditions.checkArgument(partNames.length == partTypes.length,
+        "Partition Names, " + Arrays.toString(partNames) + " don't match partition Types, "
+        + Arrays.toString(partTypes));
+
+    Map<String, String> typeMap = new HashMap();
+    for (int i = 0; i < partNames.length; i++) {
+      String previousValue = typeMap.put(partNames[i], partTypes[i]);
+      Preconditions.checkArgument(previousValue == null, "Partition columns configuration is inconsistent. "
+          + "There are duplicates in partition column names: " + partNames);
+    }
 
+    StringBuilder partNamesBuf = new StringBuilder();
+    StringBuilder partTypesBuf = new StringBuilder();
+    for (String partName : partColNames) {
+      partNamesBuf.append(partName).append('/');
+      String partType = typeMap.get(partName);
+      if (partType == null) {
+        throw new RuntimeException("Type information for partition column " + partName + " is missing.");
+      }
+      partTypesBuf.append(partType).append(':');
+    }
+    partNamesBuf.setLength(partNamesBuf.length() - 1);
+    partTypesBuf.setLength(partTypesBuf.length() - 1);
+
+    properties.setProperty(
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS,
+        partNamesBuf.toString());
+    properties.setProperty(
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES,
+        partTypesBuf.toString());
+  }
   private GenMapRedUtils() {
     // prevent instantiation
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ff1f5b1a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/TestGenMapRedUtilsUsePartitionColumnsNegative.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/TestGenMapRedUtilsUsePartitionColumnsNegative.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/TestGenMapRedUtilsUsePartitionColumnsNegative.java
new file mode 100644
index 0000000..153061f
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/TestGenMapRedUtilsUsePartitionColumnsNegative.java
@@ -0,0 +1,73 @@
+/**
+ * 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.optimizer;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Properties;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.junit.Test;
+
+public class TestGenMapRedUtilsUsePartitionColumnsNegative {
+
+  @Test(expected = NullPointerException.class)
+  public void testUsePartitionColumnsNoPartColNames() {
+    Properties p = new Properties();
+    GenMapRedUtils.usePartitionColumns(p, Arrays.asList("p1"));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testUsePartitionColumnsNamesTypesMismatch() {
+    Properties p = new Properties();
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "p1/p2");
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES, "t1");
+    GenMapRedUtils.usePartitionColumns(p, Arrays.asList("p1"));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testUsePartitionColumnsNoPartitionsToRetain() {
+    Properties p = new Properties();
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "p1");
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES, "t1");
+    GenMapRedUtils.usePartitionColumns(p, Collections.EMPTY_LIST);
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testUsePartitionColumnsWrongPartColName() {
+    Properties p = new Properties();
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "p1");
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES, "t1");
+    GenMapRedUtils.usePartitionColumns(p, Arrays.asList("p2"));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testUsePartitionColumnsDuplicatePartColNameInArgument() {
+    Properties p = new Properties();
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "p1/p2");
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES, "t1:t2");
+    GenMapRedUtils.usePartitionColumns(p, Arrays.asList("p1","p2","p1"));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testUsePartitionColumnsDuplicatePartColNameInConfiguration() {
+    Properties p = new Properties();
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "p1/p2/p1");
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES, "t1:t2:t3");
+    GenMapRedUtils.usePartitionColumns(p, Arrays.asList("p1"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ff1f5b1a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/TestGenMapRedUtilsUsePartitionColumnsPositive.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/TestGenMapRedUtilsUsePartitionColumnsPositive.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/TestGenMapRedUtilsUsePartitionColumnsPositive.java
new file mode 100644
index 0000000..9bcca66
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/TestGenMapRedUtilsUsePartitionColumnsPositive.java
@@ -0,0 +1,61 @@
+/**
+ * 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.optimizer;
+
+import java.util.Arrays;
+import java.util.Properties;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class TestGenMapRedUtilsUsePartitionColumnsPositive {
+
+  @Parameterized.Parameters(name = "{index}: updatePartitions({2})")
+  public static Iterable<Object[]> testCases() {
+    return Arrays.asList(new Object[][]{
+      {"p1/p2/p3","t1:t2:t3","p2","p2","t2"},
+      {"p1/p2/p3","t1:t2:t3","p2,p3","p2/p3","t2:t3"},
+      {"p1/p2/p3","t1:t2:t3","p1,p2,p3","p1/p2/p3","t1:t2:t3"},
+      {"p1/p2/p3","t1:t2:t3","p1,p3","p1/p3","t1:t3"},
+      {"p1","t1","p1","p1","t1"},
+      {"p1/p2/p3","t1:t2:t3","p3,p2,p1","p3/p2/p1","t3:t2:t1"}
+    });
+  }
+
+  @Parameterized.Parameter(0) public String inPartColNames;
+  @Parameterized.Parameter(1) public String inPartColTypes;
+  @Parameterized.Parameter(2) public String partNamesToRetain;
+  @Parameterized.Parameter(3) public String expectedPartColNames;
+  @Parameterized.Parameter(4) public String expectedPartColTypes;
+
+  @Test
+  public void testUsePartitionColumns() {
+    Properties p = new Properties();
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, inPartColNames);
+    p.setProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES, inPartColTypes);
+    GenMapRedUtils.usePartitionColumns(p, Arrays.asList(partNamesToRetain.split(",")));
+    String actualNames = p.getProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS);
+    String actualTypes = p.getProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES);
+    assertEquals(expectedPartColNames, actualNames);
+    assertEquals(expectedPartColTypes, actualTypes);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ff1f5b1a/ql/src/test/queries/clientpositive/dynpart_merge.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/dynpart_merge.q b/ql/src/test/queries/clientpositive/dynpart_merge.q
new file mode 100644
index 0000000..26f4de7
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/dynpart_merge.q
@@ -0,0 +1,28 @@
+set hive.exec.dynamic.partition=true;
+set hive.exec.dynamic.partition.mode=strict;
+set hive.optimize.sort.dynamic.partition=false;
+set hive.merge.mapfiles=true;
+set hive.merge.mapredfiles=true;
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+create external table sdp (
+  dataint bigint,
+  hour int,
+  req string,
+  cid string,
+  caid string
+)
+row format delimited
+fields terminated by ',';
+
+load data local inpath '../../data/files/dynpartdata1.txt' into table sdp;
+load data local inpath '../../data/files/dynpartdata2.txt' into table sdp;
+
+create table tdp (cid string, caid string)
+partitioned by (dataint bigint, hour int, req string);
+
+insert overwrite table tdp partition (dataint=20150316, hour=16, req)
+select cid, caid, req from sdp where dataint=20150316 and hour=16;
+
+select * from tdp order by caid;
+show partitions tdp;

http://git-wip-us.apache.org/repos/asf/hive/blob/ff1f5b1a/ql/src/test/results/clientpositive/dynpart_merge.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_merge.q.out b/ql/src/test/results/clientpositive/dynpart_merge.q.out
new file mode 100644
index 0000000..1c6f556
--- /dev/null
+++ b/ql/src/test/results/clientpositive/dynpart_merge.q.out
@@ -0,0 +1,99 @@
+PREHOOK: query: create external table sdp (
+  dataint bigint,
+  hour int,
+  req string,
+  cid string,
+  caid string
+)
+row format delimited
+fields terminated by ','
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@sdp
+POSTHOOK: query: create external table sdp (
+  dataint bigint,
+  hour int,
+  req string,
+  cid string,
+  caid string
+)
+row format delimited
+fields terminated by ','
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@sdp
+PREHOOK: query: load data local inpath '../../data/files/dynpartdata1.txt' into table sdp
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@sdp
+POSTHOOK: query: load data local inpath '../../data/files/dynpartdata1.txt' into table sdp
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@sdp
+PREHOOK: query: load data local inpath '../../data/files/dynpartdata2.txt' into table sdp
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@sdp
+POSTHOOK: query: load data local inpath '../../data/files/dynpartdata2.txt' into table sdp
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@sdp
+PREHOOK: query: create table tdp (cid string, caid string)
+partitioned by (dataint bigint, hour int, req string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tdp
+POSTHOOK: query: create table tdp (cid string, caid string)
+partitioned by (dataint bigint, hour int, req string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tdp
+PREHOOK: query: insert overwrite table tdp partition (dataint=20150316, hour=16, req)
+select cid, caid, req from sdp where dataint=20150316 and hour=16
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sdp
+PREHOOK: Output: default@tdp@dataint=20150316/hour=16
+POSTHOOK: query: insert overwrite table tdp partition (dataint=20150316, hour=16, req)
+select cid, caid, req from sdp where dataint=20150316 and hour=16
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sdp
+POSTHOOK: Output: default@tdp@dataint=20150316/hour=16/req=reqA
+POSTHOOK: Output: default@tdp@dataint=20150316/hour=16/req=reqB
+POSTHOOK: Output: default@tdp@dataint=20150316/hour=16/req=reqD
+POSTHOOK: Lineage: tdp PARTITION(dataint=20150316,hour=16,req=reqA).caid SIMPLE [(sdp)sdp.FieldSchema(name:caid, type:string, comment:null), ]
+POSTHOOK: Lineage: tdp PARTITION(dataint=20150316,hour=16,req=reqA).cid SIMPLE [(sdp)sdp.FieldSchema(name:cid, type:string, comment:null), ]
+POSTHOOK: Lineage: tdp PARTITION(dataint=20150316,hour=16,req=reqB).caid SIMPLE [(sdp)sdp.FieldSchema(name:caid, type:string, comment:null), ]
+POSTHOOK: Lineage: tdp PARTITION(dataint=20150316,hour=16,req=reqB).cid SIMPLE [(sdp)sdp.FieldSchema(name:cid, type:string, comment:null), ]
+POSTHOOK: Lineage: tdp PARTITION(dataint=20150316,hour=16,req=reqD).caid SIMPLE [(sdp)sdp.FieldSchema(name:caid, type:string, comment:null), ]
+POSTHOOK: Lineage: tdp PARTITION(dataint=20150316,hour=16,req=reqD).cid SIMPLE [(sdp)sdp.FieldSchema(name:cid, type:string, comment:null), ]
+PREHOOK: query: select * from tdp order by caid
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tdp
+PREHOOK: Input: default@tdp@dataint=20150316/hour=16/req=reqA
+PREHOOK: Input: default@tdp@dataint=20150316/hour=16/req=reqB
+PREHOOK: Input: default@tdp@dataint=20150316/hour=16/req=reqD
+#### A masked pattern was here ####
+POSTHOOK: query: select * from tdp order by caid
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tdp
+POSTHOOK: Input: default@tdp@dataint=20150316/hour=16/req=reqA
+POSTHOOK: Input: default@tdp@dataint=20150316/hour=16/req=reqB
+POSTHOOK: Input: default@tdp@dataint=20150316/hour=16/req=reqD
+#### A masked pattern was here ####
+clusterIdA	cacheId1	20150316	16	reqB
+clusterIdA	cacheId1	20150316	16	reqA
+clusterIdB	cacheId2	20150316	16	reqB
+clusterIdC	cacheId3	20150316	16	reqA
+clusterIdD	cacheId4	20150316	16	reqD
+clusterIdA	cacheId5	20150316	16	reqA
+clusterIdD	cacheId8	20150316	16	reqA
+clusterIdB	cacheId9	20150316	16	reqD
+PREHOOK: query: show partitions tdp
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@tdp
+POSTHOOK: query: show partitions tdp
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@tdp
+dataint=20150316/hour=16/req=reqA
+dataint=20150316/hour=16/req=reqB
+dataint=20150316/hour=16/req=reqD

http://git-wip-us.apache.org/repos/asf/hive/blob/ff1f5b1a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.7.out
index d223234..c3ede05 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.7.out
@@ -540,7 +540,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                       name default.list_bucketing_dynamic_part
                       partition_columns hr
-                      partition_columns.types string:string
+                      partition_columns.types string
                       serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                       serialization.format 1
                       serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -677,7 +677,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.list_bucketing_dynamic_part
                 partition_columns hr
-                partition_columns.types string:string
+                partition_columns.types string
                 serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -709,7 +709,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
               name default.list_bucketing_dynamic_part
               partition_columns hr
-              partition_columns.types string:string
+              partition_columns.types string
               serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -726,7 +726,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.list_bucketing_dynamic_part
                 partition_columns hr
-                partition_columns.types string:string
+                partition_columns.types string
                 serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -758,7 +758,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
               name default.list_bucketing_dynamic_part
               partition_columns hr
-              partition_columns.types string:string
+              partition_columns.types string
               serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -775,7 +775,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.list_bucketing_dynamic_part
                 partition_columns hr
-                partition_columns.types string:string
+                partition_columns.types string
                 serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe

http://git-wip-us.apache.org/repos/asf/hive/blob/ff1f5b1a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.8.out
index f884ace..16a6e72 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.8.out
@@ -540,7 +540,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                       name default.list_bucketing_dynamic_part
                       partition_columns hr
-                      partition_columns.types string:string
+                      partition_columns.types string
                       serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                       serialization.format 1
                       serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -677,7 +677,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.list_bucketing_dynamic_part
                 partition_columns hr
-                partition_columns.types string:string
+                partition_columns.types string
                 serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -709,7 +709,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
               name default.list_bucketing_dynamic_part
               partition_columns hr
-              partition_columns.types string:string
+              partition_columns.types string
               serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -726,7 +726,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.list_bucketing_dynamic_part
                 partition_columns hr
-                partition_columns.types string:string
+                partition_columns.types string
                 serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -758,7 +758,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
               name default.list_bucketing_dynamic_part
               partition_columns hr
-              partition_columns.types string:string
+              partition_columns.types string
               serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -775,7 +775,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.list_bucketing_dynamic_part
                 partition_columns hr
-                partition_columns.types string:string
+                partition_columns.types string
                 serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe

http://git-wip-us.apache.org/repos/asf/hive/blob/ff1f5b1a/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out b/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out
index 541944d..7bf4a21 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out
@@ -486,7 +486,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                       name default.list_bucketing_dynamic_part
                       partition_columns hr
-                      partition_columns.types string:string
+                      partition_columns.types string
                       serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                       serialization.format 1
                       serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -623,7 +623,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.list_bucketing_dynamic_part
                 partition_columns hr
-                partition_columns.types string:string
+                partition_columns.types string
                 serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -655,7 +655,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
               name default.list_bucketing_dynamic_part
               partition_columns hr
-              partition_columns.types string:string
+              partition_columns.types string
               serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -672,7 +672,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.list_bucketing_dynamic_part
                 partition_columns hr
-                partition_columns.types string:string
+                partition_columns.types string
                 serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -704,7 +704,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
               name default.list_bucketing_dynamic_part
               partition_columns hr
-              partition_columns.types string:string
+              partition_columns.types string
               serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
@@ -721,7 +721,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
                 name default.list_bucketing_dynamic_part
                 partition_columns hr
-                partition_columns.types string:string
+                partition_columns.types string
                 serialization.ddl struct list_bucketing_dynamic_part { string key, string value}
                 serialization.format 1
                 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe


[05/50] [abbrv] hive git commit: HIVE-11761: DoubleWritable hashcode for GroupBy is not properly generated (Aihua Xu, reviewed by Chao Sun)

Posted by xu...@apache.org.
HIVE-11761: DoubleWritable hashcode for GroupBy is not properly generated (Aihua Xu, reviewed by Chao Sun)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/70144073
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/70144073
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/70144073

Branch: refs/heads/spark
Commit: 70144073466fb72b5903ede37d65b159d44a367a
Parents: 7a71e50
Author: Aihua Xu <ai...@gmail.com>
Authored: Thu Sep 10 10:18:24 2015 -0700
Committer: Chao Sun <su...@apache.org>
Committed: Thu Sep 10 10:18:33 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java   | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/70144073/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
index 22bd951..1c409a2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.Object
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.Text;
 
 public class KeyWrapperFactory {
@@ -114,6 +115,9 @@ public class KeyWrapperFactory {
             if(element instanceof LazyDouble) {
               long v = Double.doubleToLongBits(((LazyDouble)element).getWritableObject().get());
               hashcode = hashcode + (int) (v ^ (v >>> 32));
+            } else if (element instanceof DoubleWritable){
+              long v = Double.doubleToLongBits(((DoubleWritable)element).get());
+              hashcode = hashcode + (int) (v ^ (v >>> 32));
             } else {
               hashcode = hashcode + element.hashCode();
             }


[40/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_2.q.out b/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
index 8156789..57fcc3c 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
@@ -335,245 +335,248 @@ Stage-0
       limit:100
       Stage-1
          Reducer 5
-         File Output Operator [FS_69]
+         File Output Operator [FS_68]
             compressed:false
             Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
             table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
-            Limit [LIM_68]
+            Limit [LIM_67]
                Number of rows:100
                Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
-               Select Operator [SEL_67]
+               Select Operator [SEL_66]
                |  outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
                |  Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
                |<-Reducer 4 [SIMPLE_EDGE]
-                  Reduce Output Operator [RS_66]
+                  Reduce Output Operator [RS_65]
                      key expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string)
                      sort order:+++
                      Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
                      value expressions:_col3 (type: bigint), _col4 (type: bigint), _col5 (type: bigint)
-                     Group By Operator [GBY_64]
-                     |  aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
-                     |  keys:KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
-                     |  outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                     |  Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
-                     |<-Reducer 3 [SIMPLE_EDGE]
-                        Reduce Output Operator [RS_63]
-                           key expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string)
-                           Map-reduce partition columns:_col0 (type: string), _col1 (type: string), _col2 (type: string)
-                           sort order:+++
-                           Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
-                           value expressions:_col3 (type: bigint), _col4 (type: bigint), _col5 (type: bigint)
-                           Group By Operator [GBY_62]
-                              aggregations:["count(_col3)","count(_col4)","count(_col5)"]
-                              keys:_col0 (type: string), _col1 (type: string), _col2 (type: string)
-                              outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                     Select Operator [SEL_64]
+                        outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                        Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
+                        Group By Operator [GBY_63]
+                        |  aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
+                        |  keys:KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
+                        |  outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                        |  Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
+                        |<-Reducer 3 [SIMPLE_EDGE]
+                           Reduce Output Operator [RS_62]
+                              key expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string)
+                              Map-reduce partition columns:_col0 (type: string), _col1 (type: string), _col2 (type: string)
+                              sort order:+++
                               Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
-                              Select Operator [SEL_60]
+                              value expressions:_col3 (type: bigint), _col4 (type: bigint), _col5 (type: bigint)
+                              Group By Operator [GBY_61]
+                                 aggregations:["count(_col13)","count(_col21)","count(_col3)"]
+                                 keys:_col2 (type: string), _col12 (type: string), _col20 (type: string)
                                  outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
                                  Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
-                                 Merge Join Operator [MERGEJOIN_111]
-                                 |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |  keys:{"0":"_col1 (type: string), _col3 (type: string)","1":"_col15 (type: string), _col17 (type: string)"}
-                                 |  outputColumnNames:["_col2","_col3","_col12","_col13","_col20","_col21"]
-                                 |  Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
-                                 |<-Reducer 11 [SIMPLE_EDGE]
-                                 |  Reduce Output Operator [RS_58]
-                                 |     key expressions:_col15 (type: string), _col17 (type: string)
-                                 |     Map-reduce partition columns:_col15 (type: string), _col17 (type: string)
-                                 |     sort order:++
-                                 |     Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
-                                 |     value expressions:_col6 (type: string), _col7 (type: string), _col14 (type: string)
-                                 |     Select Operator [SEL_49]
-                                 |        outputColumnNames:["_col14","_col15","_col17","_col6","_col7"]
-                                 |        Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
-                                 |        Merge Join Operator [MERGEJOIN_110]
-                                 |        |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |        |  keys:{"0":"_col4 (type: string), _col6 (type: string)","1":"_col2 (type: string), _col4 (type: string)"}
-                                 |        |  outputColumnNames:["_col2","_col3","_col14","_col15","_col17"]
-                                 |        |  Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
-                                 |        |<-Reducer 10 [SIMPLE_EDGE]
-                                 |        |  Reduce Output Operator [RS_45]
-                                 |        |     key expressions:_col4 (type: string), _col6 (type: string)
-                                 |        |     Map-reduce partition columns:_col4 (type: string), _col6 (type: string)
-                                 |        |     sort order:++
-                                 |        |     Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
-                                 |        |     value expressions:_col2 (type: string), _col3 (type: string)
-                                 |        |     Merge Join Operator [MERGEJOIN_108]
-                                 |        |     |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |        |     |  keys:{"0":"_col3 (type: string)","1":"_col1 (type: string)"}
-                                 |        |     |  outputColumnNames:["_col2","_col3","_col4","_col6"]
-                                 |        |     |  Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
-                                 |        |     |<-Map 14 [SIMPLE_EDGE]
-                                 |        |     |  Reduce Output Operator [RS_42]
-                                 |        |     |     key expressions:_col1 (type: string)
-                                 |        |     |     Map-reduce partition columns:_col1 (type: string)
-                                 |        |     |     sort order:+
-                                 |        |     |     Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
-                                 |        |     |     Select Operator [SEL_16]
-                                 |        |     |        outputColumnNames:["_col1"]
-                                 |        |     |        Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
-                                 |        |     |        Filter Operator [FIL_102]
-                                 |        |     |           predicate:((key = 'src1key') and value is not null) (type: boolean)
-                                 |        |     |           Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
-                                 |        |     |           TableScan [TS_14]
-                                 |        |     |              alias:src1
-                                 |        |     |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                                 |        |     |<-Reducer 9 [SIMPLE_EDGE]
-                                 |        |        Reduce Output Operator [RS_40]
-                                 |        |           key expressions:_col3 (type: string)
-                                 |        |           Map-reduce partition columns:_col3 (type: string)
-                                 |        |           sort order:+
-                                 |        |           Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                                 |        |           value expressions:_col2 (type: string), _col4 (type: string), _col6 (type: string)
-                                 |        |           Merge Join Operator [MERGEJOIN_107]
-                                 |        |           |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |        |           |  keys:{"0":"_col2 (type: string)","1":"_col0 (type: string)"}
-                                 |        |           |  outputColumnNames:["_col2","_col3","_col4","_col6"]
-                                 |        |           |  Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                                 |        |           |<-Map 13 [SIMPLE_EDGE]
-                                 |        |           |  Reduce Output Operator [RS_37]
-                                 |        |           |     key expressions:_col0 (type: string)
-                                 |        |           |     Map-reduce partition columns:_col0 (type: string)
-                                 |        |           |     sort order:+
-                                 |        |           |     Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                 |        |           |     Select Operator [SEL_13]
-                                 |        |           |        outputColumnNames:["_col0"]
-                                 |        |           |        Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                 |        |           |        Filter Operator [FIL_101]
-                                 |        |           |           predicate:((value = 'd1value') and key is not null) (type: boolean)
-                                 |        |           |           Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                 |        |           |           TableScan [TS_11]
-                                 |        |           |              alias:d1
-                                 |        |           |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                 |        |           |<-Reducer 8 [SIMPLE_EDGE]
-                                 |        |              Reduce Output Operator [RS_35]
-                                 |        |                 key expressions:_col2 (type: string)
-                                 |        |                 Map-reduce partition columns:_col2 (type: string)
-                                 |        |                 sort order:+
-                                 |        |                 Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                                 |        |                 value expressions:_col3 (type: string), _col4 (type: string), _col6 (type: string)
-                                 |        |                 Merge Join Operator [MERGEJOIN_106]
-                                 |        |                 |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |        |                 |  keys:{"0":"_col1 (type: string)","1":"_col3 (type: string)"}
-                                 |        |                 |  outputColumnNames:["_col2","_col3","_col4","_col6"]
-                                 |        |                 |  Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                                 |        |                 |<-Map 12 [SIMPLE_EDGE]
-                                 |        |                 |  Reduce Output Operator [RS_32]
-                                 |        |                 |     key expressions:_col3 (type: string)
-                                 |        |                 |     Map-reduce partition columns:_col3 (type: string)
-                                 |        |                 |     sort order:+
-                                 |        |                 |     Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                 |        |                 |     value expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string), _col4 (type: string)
-                                 |        |                 |     Select Operator [SEL_10]
-                                 |        |                 |        outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
-                                 |        |                 |        Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                 |        |                 |        Filter Operator [FIL_100]
-                                 |        |                 |           predicate:((((((v3 = 'ssv3') and v2 is not null) and k1 is not null) and v1 is not null) and k2 is not null) and k3 is not null) (type: boolean)
-                                 |        |                 |           Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                 |        |                 |           TableScan [TS_8]
-                                 |        |                 |              alias:ss
-                                 |        |                 |              Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
-                                 |        |                 |<-Map 7 [SIMPLE_EDGE]
-                                 |        |                    Reduce Output Operator [RS_30]
-                                 |        |                       key expressions:_col1 (type: string)
-                                 |        |                       Map-reduce partition columns:_col1 (type: string)
-                                 |        |                       sort order:+
-                                 |        |                       Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                 |        |                       Select Operator [SEL_7]
-                                 |        |                          outputColumnNames:["_col1"]
-                                 |        |                          Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                 |        |                          Filter Operator [FIL_99]
-                                 |        |                             predicate:((key = 'srcpartkey') and value is not null) (type: boolean)
-                                 |        |                             Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                 |        |                             TableScan [TS_5]
-                                 |        |                                alias:srcpart
-                                 |        |                                Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                                 |        |<-Reducer 16 [SIMPLE_EDGE]
-                                 |           Reduce Output Operator [RS_47]
-                                 |              key expressions:_col2 (type: string), _col4 (type: string)
-                                 |              Map-reduce partition columns:_col2 (type: string), _col4 (type: string)
-                                 |              sort order:++
-                                 |              Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-                                 |              value expressions:_col3 (type: string), _col5 (type: string)
-                                 |              Merge Join Operator [MERGEJOIN_109]
-                                 |              |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |              |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
-                                 |              |  outputColumnNames:["_col2","_col3","_col4","_col5"]
-                                 |              |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-                                 |              |<-Map 15 [SIMPLE_EDGE]
-                                 |              |  Reduce Output Operator [RS_24]
-                                 |              |     key expressions:_col0 (type: string)
-                                 |              |     Map-reduce partition columns:_col0 (type: string)
-                                 |              |     sort order:+
-                                 |              |     Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                 |              |     value expressions:_col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
-                                 |              |     Select Operator [SEL_19]
-                                 |              |        outputColumnNames:["_col0","_col2","_col3","_col4","_col5"]
-                                 |              |        Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                 |              |        Filter Operator [FIL_103]
-                                 |              |           predicate:((((((v1 = 'srv1') and k1 is not null) and k2 is not null) and k3 is not null) and v2 is not null) and v3 is not null) (type: boolean)
-                                 |              |           Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                 |              |           TableScan [TS_17]
-                                 |              |              alias:sr
-                                 |              |              Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
-                                 |              |<-Map 17 [SIMPLE_EDGE]
-                                 |                 Reduce Output Operator [RS_26]
-                                 |                    key expressions:_col0 (type: string)
-                                 |                    Map-reduce partition columns:_col0 (type: string)
-                                 |                    sort order:+
-                                 |                    Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                 |                    Select Operator [SEL_22]
-                                 |                       outputColumnNames:["_col0"]
-                                 |                       Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                 |                       Filter Operator [FIL_104]
-                                 |                          predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
-                                 |                          Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                 |                          TableScan [TS_20]
-                                 |                             alias:d1
-                                 |                             Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                 |<-Reducer 2 [SIMPLE_EDGE]
-                                    Reduce Output Operator [RS_56]
-                                       key expressions:_col1 (type: string), _col3 (type: string)
-                                       Map-reduce partition columns:_col1 (type: string), _col3 (type: string)
-                                       sort order:++
-                                       Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-                                       value expressions:_col2 (type: string)
-                                       Merge Join Operator [MERGEJOIN_105]
-                                       |  condition map:[{"":"Inner Join 0 to 1"}]
-                                       |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
-                                       |  outputColumnNames:["_col1","_col2","_col3"]
-                                       |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-                                       |<-Map 1 [SIMPLE_EDGE]
-                                       |  Reduce Output Operator [RS_51]
-                                       |     key expressions:_col0 (type: string)
-                                       |     Map-reduce partition columns:_col0 (type: string)
-                                       |     sort order:+
-                                       |     Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
-                                       |     value expressions:_col1 (type: string), _col2 (type: string), _col3 (type: string)
-                                       |     Select Operator [SEL_1]
-                                       |        outputColumnNames:["_col0","_col1","_col2","_col3"]
-                                       |        Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
-                                       |        Filter Operator [FIL_97]
-                                       |           predicate:((k1 is not null and v2 is not null) and v3 is not null) (type: boolean)
-                                       |           Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
-                                       |           TableScan [TS_0]
-                                       |              alias:cs
-                                       |              Statistics:Num rows: 170 Data size: 5890 Basic stats: COMPLETE Column stats: NONE
-                                       |<-Map 6 [SIMPLE_EDGE]
-                                          Reduce Output Operator [RS_53]
-                                             key expressions:_col0 (type: string)
-                                             Map-reduce partition columns:_col0 (type: string)
-                                             sort order:+
-                                             Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                             Select Operator [SEL_4]
-                                                outputColumnNames:["_col0"]
+                                 Select Operator [SEL_60]
+                                    outputColumnNames:["_col2","_col12","_col20","_col13","_col21","_col3"]
+                                    Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
+                                    Merge Join Operator [MERGEJOIN_110]
+                                    |  condition map:[{"":"Inner Join 0 to 1"}]
+                                    |  keys:{"0":"_col1 (type: string), _col3 (type: string)","1":"_col15 (type: string), _col17 (type: string)"}
+                                    |  outputColumnNames:["_col2","_col3","_col12","_col13","_col20","_col21"]
+                                    |  Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
+                                    |<-Reducer 11 [SIMPLE_EDGE]
+                                    |  Reduce Output Operator [RS_58]
+                                    |     key expressions:_col15 (type: string), _col17 (type: string)
+                                    |     Map-reduce partition columns:_col15 (type: string), _col17 (type: string)
+                                    |     sort order:++
+                                    |     Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
+                                    |     value expressions:_col6 (type: string), _col7 (type: string), _col14 (type: string)
+                                    |     Select Operator [SEL_49]
+                                    |        outputColumnNames:["_col14","_col15","_col17","_col6","_col7"]
+                                    |        Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
+                                    |        Merge Join Operator [MERGEJOIN_109]
+                                    |        |  condition map:[{"":"Inner Join 0 to 1"}]
+                                    |        |  keys:{"0":"_col4 (type: string), _col6 (type: string)","1":"_col2 (type: string), _col4 (type: string)"}
+                                    |        |  outputColumnNames:["_col2","_col3","_col14","_col15","_col17"]
+                                    |        |  Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
+                                    |        |<-Reducer 10 [SIMPLE_EDGE]
+                                    |        |  Reduce Output Operator [RS_45]
+                                    |        |     key expressions:_col4 (type: string), _col6 (type: string)
+                                    |        |     Map-reduce partition columns:_col4 (type: string), _col6 (type: string)
+                                    |        |     sort order:++
+                                    |        |     Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
+                                    |        |     value expressions:_col2 (type: string), _col3 (type: string)
+                                    |        |     Merge Join Operator [MERGEJOIN_107]
+                                    |        |     |  condition map:[{"":"Inner Join 0 to 1"}]
+                                    |        |     |  keys:{"0":"_col3 (type: string)","1":"_col1 (type: string)"}
+                                    |        |     |  outputColumnNames:["_col2","_col3","_col4","_col6"]
+                                    |        |     |  Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
+                                    |        |     |<-Map 14 [SIMPLE_EDGE]
+                                    |        |     |  Reduce Output Operator [RS_42]
+                                    |        |     |     key expressions:_col1 (type: string)
+                                    |        |     |     Map-reduce partition columns:_col1 (type: string)
+                                    |        |     |     sort order:+
+                                    |        |     |     Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
+                                    |        |     |     Select Operator [SEL_16]
+                                    |        |     |        outputColumnNames:["_col1"]
+                                    |        |     |        Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
+                                    |        |     |        Filter Operator [FIL_101]
+                                    |        |     |           predicate:((key = 'src1key') and value is not null) (type: boolean)
+                                    |        |     |           Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
+                                    |        |     |           TableScan [TS_14]
+                                    |        |     |              alias:src1
+                                    |        |     |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                                    |        |     |<-Reducer 9 [SIMPLE_EDGE]
+                                    |        |        Reduce Output Operator [RS_40]
+                                    |        |           key expressions:_col3 (type: string)
+                                    |        |           Map-reduce partition columns:_col3 (type: string)
+                                    |        |           sort order:+
+                                    |        |           Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+                                    |        |           value expressions:_col2 (type: string), _col4 (type: string), _col6 (type: string)
+                                    |        |           Merge Join Operator [MERGEJOIN_106]
+                                    |        |           |  condition map:[{"":"Inner Join 0 to 1"}]
+                                    |        |           |  keys:{"0":"_col2 (type: string)","1":"_col0 (type: string)"}
+                                    |        |           |  outputColumnNames:["_col2","_col3","_col4","_col6"]
+                                    |        |           |  Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+                                    |        |           |<-Map 13 [SIMPLE_EDGE]
+                                    |        |           |  Reduce Output Operator [RS_37]
+                                    |        |           |     key expressions:_col0 (type: string)
+                                    |        |           |     Map-reduce partition columns:_col0 (type: string)
+                                    |        |           |     sort order:+
+                                    |        |           |     Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                    |        |           |     Select Operator [SEL_13]
+                                    |        |           |        outputColumnNames:["_col0"]
+                                    |        |           |        Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                    |        |           |        Filter Operator [FIL_100]
+                                    |        |           |           predicate:((value = 'd1value') and key is not null) (type: boolean)
+                                    |        |           |           Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                    |        |           |           TableScan [TS_11]
+                                    |        |           |              alias:d1
+                                    |        |           |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                    |        |           |<-Reducer 8 [SIMPLE_EDGE]
+                                    |        |              Reduce Output Operator [RS_35]
+                                    |        |                 key expressions:_col2 (type: string)
+                                    |        |                 Map-reduce partition columns:_col2 (type: string)
+                                    |        |                 sort order:+
+                                    |        |                 Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                                    |        |                 value expressions:_col3 (type: string), _col4 (type: string), _col6 (type: string)
+                                    |        |                 Merge Join Operator [MERGEJOIN_105]
+                                    |        |                 |  condition map:[{"":"Inner Join 0 to 1"}]
+                                    |        |                 |  keys:{"0":"_col1 (type: string)","1":"_col3 (type: string)"}
+                                    |        |                 |  outputColumnNames:["_col2","_col3","_col4","_col6"]
+                                    |        |                 |  Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                                    |        |                 |<-Map 12 [SIMPLE_EDGE]
+                                    |        |                 |  Reduce Output Operator [RS_32]
+                                    |        |                 |     key expressions:_col3 (type: string)
+                                    |        |                 |     Map-reduce partition columns:_col3 (type: string)
+                                    |        |                 |     sort order:+
+                                    |        |                 |     Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                    |        |                 |     value expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string), _col4 (type: string)
+                                    |        |                 |     Select Operator [SEL_10]
+                                    |        |                 |        outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
+                                    |        |                 |        Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                    |        |                 |        Filter Operator [FIL_99]
+                                    |        |                 |           predicate:((((((v3 = 'ssv3') and v2 is not null) and k1 is not null) and v1 is not null) and k2 is not null) and k3 is not null) (type: boolean)
+                                    |        |                 |           Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                    |        |                 |           TableScan [TS_8]
+                                    |        |                 |              alias:ss
+                                    |        |                 |              Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
+                                    |        |                 |<-Map 7 [SIMPLE_EDGE]
+                                    |        |                    Reduce Output Operator [RS_30]
+                                    |        |                       key expressions:_col1 (type: string)
+                                    |        |                       Map-reduce partition columns:_col1 (type: string)
+                                    |        |                       sort order:+
+                                    |        |                       Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                    |        |                       Select Operator [SEL_7]
+                                    |        |                          outputColumnNames:["_col1"]
+                                    |        |                          Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                    |        |                          Filter Operator [FIL_98]
+                                    |        |                             predicate:((key = 'srcpartkey') and value is not null) (type: boolean)
+                                    |        |                             Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                    |        |                             TableScan [TS_5]
+                                    |        |                                alias:srcpart
+                                    |        |                                Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
+                                    |        |<-Reducer 16 [SIMPLE_EDGE]
+                                    |           Reduce Output Operator [RS_47]
+                                    |              key expressions:_col2 (type: string), _col4 (type: string)
+                                    |              Map-reduce partition columns:_col2 (type: string), _col4 (type: string)
+                                    |              sort order:++
+                                    |              Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                                    |              value expressions:_col3 (type: string), _col5 (type: string)
+                                    |              Merge Join Operator [MERGEJOIN_108]
+                                    |              |  condition map:[{"":"Inner Join 0 to 1"}]
+                                    |              |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
+                                    |              |  outputColumnNames:["_col2","_col3","_col4","_col5"]
+                                    |              |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                                    |              |<-Map 15 [SIMPLE_EDGE]
+                                    |              |  Reduce Output Operator [RS_24]
+                                    |              |     key expressions:_col0 (type: string)
+                                    |              |     Map-reduce partition columns:_col0 (type: string)
+                                    |              |     sort order:+
+                                    |              |     Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                    |              |     value expressions:_col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                                    |              |     Select Operator [SEL_19]
+                                    |              |        outputColumnNames:["_col0","_col2","_col3","_col4","_col5"]
+                                    |              |        Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                    |              |        Filter Operator [FIL_102]
+                                    |              |           predicate:((((((v1 = 'srv1') and k1 is not null) and k2 is not null) and k3 is not null) and v2 is not null) and v3 is not null) (type: boolean)
+                                    |              |           Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                    |              |           TableScan [TS_17]
+                                    |              |              alias:sr
+                                    |              |              Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
+                                    |              |<-Map 17 [SIMPLE_EDGE]
+                                    |                 Reduce Output Operator [RS_26]
+                                    |                    key expressions:_col0 (type: string)
+                                    |                    Map-reduce partition columns:_col0 (type: string)
+                                    |                    sort order:+
+                                    |                    Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                    |                    Select Operator [SEL_22]
+                                    |                       outputColumnNames:["_col0"]
+                                    |                       Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                    |                       Filter Operator [FIL_103]
+                                    |                          predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
+                                    |                          Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                    |                          TableScan [TS_20]
+                                    |                             alias:d1
+                                    |                             Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                    |<-Reducer 2 [SIMPLE_EDGE]
+                                       Reduce Output Operator [RS_56]
+                                          key expressions:_col1 (type: string), _col3 (type: string)
+                                          Map-reduce partition columns:_col1 (type: string), _col3 (type: string)
+                                          sort order:++
+                                          Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                                          value expressions:_col2 (type: string)
+                                          Merge Join Operator [MERGEJOIN_104]
+                                          |  condition map:[{"":"Inner Join 0 to 1"}]
+                                          |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
+                                          |  outputColumnNames:["_col1","_col2","_col3"]
+                                          |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                                          |<-Map 1 [SIMPLE_EDGE]
+                                          |  Reduce Output Operator [RS_51]
+                                          |     key expressions:_col0 (type: string)
+                                          |     Map-reduce partition columns:_col0 (type: string)
+                                          |     sort order:+
+                                          |     Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
+                                          |     value expressions:_col1 (type: string), _col2 (type: string), _col3 (type: string)
+                                          |     Select Operator [SEL_1]
+                                          |        outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                          |        Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
+                                          |        Filter Operator [FIL_96]
+                                          |           predicate:((k1 is not null and v2 is not null) and v3 is not null) (type: boolean)
+                                          |           Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
+                                          |           TableScan [TS_0]
+                                          |              alias:cs
+                                          |              Statistics:Num rows: 170 Data size: 5890 Basic stats: COMPLETE Column stats: NONE
+                                          |<-Map 6 [SIMPLE_EDGE]
+                                             Reduce Output Operator [RS_53]
+                                                key expressions:_col0 (type: string)
+                                                Map-reduce partition columns:_col0 (type: string)
+                                                sort order:+
                                                 Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                                Filter Operator [FIL_98]
-                                                   predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
+                                                Select Operator [SEL_4]
+                                                   outputColumnNames:["_col0"]
                                                    Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                                   TableScan [TS_2]
-                                                      alias:d1
-                                                      Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                                   Filter Operator [FIL_97]
+                                                      predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
+                                                      Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                                      TableScan [TS_2]
+                                                         alias:d1
+                                                         Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 
 PREHOOK: query: explain
 SELECT x.key, z.value, y.value
@@ -1522,199 +1525,202 @@ Stage-0
       limit:100
       Stage-1
          Reducer 5
-         File Output Operator [FS_69]
+         File Output Operator [FS_68]
             compressed:false
             Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
             table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
-            Limit [LIM_68]
+            Limit [LIM_67]
                Number of rows:100
                Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
-               Select Operator [SEL_67]
+               Select Operator [SEL_66]
                |  outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
                |  Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
                |<-Reducer 4 [SIMPLE_EDGE]
-                  Reduce Output Operator [RS_66]
+                  Reduce Output Operator [RS_65]
                      key expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string)
                      sort order:+++
                      Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
                      value expressions:_col3 (type: bigint), _col4 (type: bigint), _col5 (type: bigint)
-                     Group By Operator [GBY_64]
-                     |  aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
-                     |  keys:KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
-                     |  outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                     |  Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
-                     |<-Map 3 [SIMPLE_EDGE]
-                        Reduce Output Operator [RS_63]
-                           key expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string)
-                           Map-reduce partition columns:_col0 (type: string), _col1 (type: string), _col2 (type: string)
-                           sort order:+++
-                           Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
-                           value expressions:_col3 (type: bigint), _col4 (type: bigint), _col5 (type: bigint)
-                           Group By Operator [GBY_62]
-                              aggregations:["count(_col3)","count(_col4)","count(_col5)"]
-                              keys:_col0 (type: string), _col1 (type: string), _col2 (type: string)
-                              outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                     Select Operator [SEL_64]
+                        outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                        Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
+                        Group By Operator [GBY_63]
+                        |  aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
+                        |  keys:KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
+                        |  outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                        |  Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
+                        |<-Map 3 [SIMPLE_EDGE]
+                           Reduce Output Operator [RS_62]
+                              key expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string)
+                              Map-reduce partition columns:_col0 (type: string), _col1 (type: string), _col2 (type: string)
+                              sort order:+++
                               Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
-                              Select Operator [SEL_60]
+                              value expressions:_col3 (type: bigint), _col4 (type: bigint), _col5 (type: bigint)
+                              Group By Operator [GBY_61]
+                                 aggregations:["count(_col13)","count(_col21)","count(_col3)"]
+                                 keys:_col2 (type: string), _col12 (type: string), _col20 (type: string)
                                  outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
                                  Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
-                                 Map Join Operator [MAPJOIN_111]
-                                 |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |  keys:{"Map 2":"_col1 (type: string), _col3 (type: string)","Map 3":"_col15 (type: string), _col17 (type: string)"}
-                                 |  outputColumnNames:["_col2","_col3","_col12","_col13","_col20","_col21"]
-                                 |  Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
-                                 |<-Map 2 [BROADCAST_EDGE]
-                                 |  Reduce Output Operator [RS_56]
-                                 |     key expressions:_col1 (type: string), _col3 (type: string)
-                                 |     Map-reduce partition columns:_col1 (type: string), _col3 (type: string)
-                                 |     sort order:++
-                                 |     Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-                                 |     value expressions:_col2 (type: string)
-                                 |     Map Join Operator [MAPJOIN_105]
-                                 |     |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |     |  keys:{"Map 1":"_col0 (type: string)","Map 2":"_col0 (type: string)"}
-                                 |     |  outputColumnNames:["_col1","_col2","_col3"]
-                                 |     |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-                                 |     |<-Map 1 [BROADCAST_EDGE]
-                                 |     |  Reduce Output Operator [RS_51]
-                                 |     |     key expressions:_col0 (type: string)
-                                 |     |     Map-reduce partition columns:_col0 (type: string)
-                                 |     |     sort order:+
-                                 |     |     Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
-                                 |     |     value expressions:_col1 (type: string), _col2 (type: string), _col3 (type: string)
-                                 |     |     Select Operator [SEL_1]
-                                 |     |        outputColumnNames:["_col0","_col1","_col2","_col3"]
-                                 |     |        Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
-                                 |     |        Filter Operator [FIL_97]
-                                 |     |           predicate:((k1 is not null and v2 is not null) and v3 is not null) (type: boolean)
-                                 |     |           Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
-                                 |     |           TableScan [TS_0]
-                                 |     |              alias:cs
-                                 |     |              Statistics:Num rows: 170 Data size: 5890 Basic stats: COMPLETE Column stats: NONE
-                                 |     |<-Select Operator [SEL_4]
-                                 |           outputColumnNames:["_col0"]
-                                 |           Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                 |           Filter Operator [FIL_98]
-                                 |              predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
-                                 |              Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                 |              TableScan [TS_2]
-                                 |                 alias:d1
-                                 |                 Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                 |<-Select Operator [SEL_49]
-                                       outputColumnNames:["_col14","_col15","_col17","_col6","_col7"]
-                                       Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
-                                       Map Join Operator [MAPJOIN_110]
-                                       |  condition map:[{"":"Inner Join 0 to 1"}]
-                                       |  keys:{"Map 3":"_col4 (type: string), _col6 (type: string)","Map 10":"_col2 (type: string), _col4 (type: string)"}
-                                       |  outputColumnNames:["_col2","_col3","_col14","_col15","_col17"]
-                                       |  Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
-                                       |<-Map 10 [BROADCAST_EDGE]
-                                       |  Reduce Output Operator [RS_47]
-                                       |     key expressions:_col2 (type: string), _col4 (type: string)
-                                       |     Map-reduce partition columns:_col2 (type: string), _col4 (type: string)
-                                       |     sort order:++
-                                       |     Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-                                       |     value expressions:_col3 (type: string), _col5 (type: string)
-                                       |     Map Join Operator [MAPJOIN_109]
-                                       |     |  condition map:[{"":"Inner Join 0 to 1"}]
-                                       |     |  keys:{"Map 9":"_col0 (type: string)","Map 10":"_col0 (type: string)"}
-                                       |     |  outputColumnNames:["_col2","_col3","_col4","_col5"]
-                                       |     |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-                                       |     |<-Map 9 [BROADCAST_EDGE]
-                                       |     |  Reduce Output Operator [RS_24]
-                                       |     |     key expressions:_col0 (type: string)
-                                       |     |     Map-reduce partition columns:_col0 (type: string)
-                                       |     |     sort order:+
-                                       |     |     Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                       |     |     value expressions:_col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
-                                       |     |     Select Operator [SEL_19]
-                                       |     |        outputColumnNames:["_col0","_col2","_col3","_col4","_col5"]
-                                       |     |        Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                       |     |        Filter Operator [FIL_103]
-                                       |     |           predicate:((((((v1 = 'srv1') and k1 is not null) and k2 is not null) and k3 is not null) and v2 is not null) and v3 is not null) (type: boolean)
-                                       |     |           Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                       |     |           TableScan [TS_17]
-                                       |     |              alias:sr
-                                       |     |              Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
-                                       |     |<-Select Operator [SEL_22]
-                                       |           outputColumnNames:["_col0"]
-                                       |           Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                       |           Filter Operator [FIL_104]
-                                       |              predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
-                                       |              Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                       |              TableScan [TS_20]
-                                       |                 alias:d1
-                                       |                 Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                       |<-Map Join Operator [MAPJOIN_108]
+                                 Select Operator [SEL_60]
+                                    outputColumnNames:["_col2","_col12","_col20","_col13","_col21","_col3"]
+                                    Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
+                                    Map Join Operator [MAPJOIN_110]
+                                    |  condition map:[{"":"Inner Join 0 to 1"}]
+                                    |  keys:{"Map 2":"_col1 (type: string), _col3 (type: string)","Map 3":"_col15 (type: string), _col17 (type: string)"}
+                                    |  outputColumnNames:["_col2","_col3","_col12","_col13","_col20","_col21"]
+                                    |  Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
+                                    |<-Map 2 [BROADCAST_EDGE]
+                                    |  Reduce Output Operator [RS_56]
+                                    |     key expressions:_col1 (type: string), _col3 (type: string)
+                                    |     Map-reduce partition columns:_col1 (type: string), _col3 (type: string)
+                                    |     sort order:++
+                                    |     Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                                    |     value expressions:_col2 (type: string)
+                                    |     Map Join Operator [MAPJOIN_104]
+                                    |     |  condition map:[{"":"Inner Join 0 to 1"}]
+                                    |     |  keys:{"Map 1":"_col0 (type: string)","Map 2":"_col0 (type: string)"}
+                                    |     |  outputColumnNames:["_col1","_col2","_col3"]
+                                    |     |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                                    |     |<-Map 1 [BROADCAST_EDGE]
+                                    |     |  Reduce Output Operator [RS_51]
+                                    |     |     key expressions:_col0 (type: string)
+                                    |     |     Map-reduce partition columns:_col0 (type: string)
+                                    |     |     sort order:+
+                                    |     |     Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
+                                    |     |     value expressions:_col1 (type: string), _col2 (type: string), _col3 (type: string)
+                                    |     |     Select Operator [SEL_1]
+                                    |     |        outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                    |     |        Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
+                                    |     |        Filter Operator [FIL_96]
+                                    |     |           predicate:((k1 is not null and v2 is not null) and v3 is not null) (type: boolean)
+                                    |     |           Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
+                                    |     |           TableScan [TS_0]
+                                    |     |              alias:cs
+                                    |     |              Statistics:Num rows: 170 Data size: 5890 Basic stats: COMPLETE Column stats: NONE
+                                    |     |<-Select Operator [SEL_4]
+                                    |           outputColumnNames:["_col0"]
+                                    |           Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                    |           Filter Operator [FIL_97]
+                                    |              predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
+                                    |              Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                    |              TableScan [TS_2]
+                                    |                 alias:d1
+                                    |                 Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                    |<-Select Operator [SEL_49]
+                                          outputColumnNames:["_col14","_col15","_col17","_col6","_col7"]
+                                          Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
+                                          Map Join Operator [MAPJOIN_109]
                                           |  condition map:[{"":"Inner Join 0 to 1"}]
-                                          |  keys:{"Map 3":"_col3 (type: string)","Map 8":"_col1 (type: string)"}
-                                          |  outputColumnNames:["_col2","_col3","_col4","_col6"]
-                                          |  Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
-                                          |<-Map 8 [BROADCAST_EDGE]
-                                          |  Reduce Output Operator [RS_42]
-                                          |     key expressions:_col1 (type: string)
-                                          |     Map-reduce partition columns:_col1 (type: string)
-                                          |     sort order:+
-                                          |     Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
-                                          |     Select Operator [SEL_16]
-                                          |        outputColumnNames:["_col1"]
-                                          |        Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
-                                          |        Filter Operator [FIL_102]
-                                          |           predicate:((key = 'src1key') and value is not null) (type: boolean)
-                                          |           Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
-                                          |           TableScan [TS_14]
-                                          |              alias:src1
-                                          |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                                          |  keys:{"Map 3":"_col4 (type: string), _col6 (type: string)","Map 10":"_col2 (type: string), _col4 (type: string)"}
+                                          |  outputColumnNames:["_col2","_col3","_col14","_col15","_col17"]
+                                          |  Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
+                                          |<-Map 10 [BROADCAST_EDGE]
+                                          |  Reduce Output Operator [RS_47]
+                                          |     key expressions:_col2 (type: string), _col4 (type: string)
+                                          |     Map-reduce partition columns:_col2 (type: string), _col4 (type: string)
+                                          |     sort order:++
+                                          |     Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                                          |     value expressions:_col3 (type: string), _col5 (type: string)
+                                          |     Map Join Operator [MAPJOIN_108]
+                                          |     |  condition map:[{"":"Inner Join 0 to 1"}]
+                                          |     |  keys:{"Map 9":"_col0 (type: string)","Map 10":"_col0 (type: string)"}
+                                          |     |  outputColumnNames:["_col2","_col3","_col4","_col5"]
+                                          |     |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                                          |     |<-Map 9 [BROADCAST_EDGE]
+                                          |     |  Reduce Output Operator [RS_24]
+                                          |     |     key expressions:_col0 (type: string)
+                                          |     |     Map-reduce partition columns:_col0 (type: string)
+                                          |     |     sort order:+
+                                          |     |     Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                          |     |     value expressions:_col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                                          |     |     Select Operator [SEL_19]
+                                          |     |        outputColumnNames:["_col0","_col2","_col3","_col4","_col5"]
+                                          |     |        Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                          |     |        Filter Operator [FIL_102]
+                                          |     |           predicate:((((((v1 = 'srv1') and k1 is not null) and k2 is not null) and k3 is not null) and v2 is not null) and v3 is not null) (type: boolean)
+                                          |     |           Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                          |     |           TableScan [TS_17]
+                                          |     |              alias:sr
+                                          |     |              Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
+                                          |     |<-Select Operator [SEL_22]
+                                          |           outputColumnNames:["_col0"]
+                                          |           Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                          |           Filter Operator [FIL_103]
+                                          |              predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
+                                          |              Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                          |              TableScan [TS_20]
+                                          |                 alias:d1
+                                          |                 Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                                           |<-Map Join Operator [MAPJOIN_107]
                                              |  condition map:[{"":"Inner Join 0 to 1"}]
-                                             |  keys:{"Map 3":"_col2 (type: string)","Map 7":"_col0 (type: string)"}
+                                             |  keys:{"Map 3":"_col3 (type: string)","Map 8":"_col1 (type: string)"}
                                              |  outputColumnNames:["_col2","_col3","_col4","_col6"]
-                                             |  Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                                             |<-Map 7 [BROADCAST_EDGE]
-                                             |  Reduce Output Operator [RS_37]
-                                             |     key expressions:_col0 (type: string)
-                                             |     Map-reduce partition columns:_col0 (type: string)
+                                             |  Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
+                                             |<-Map 8 [BROADCAST_EDGE]
+                                             |  Reduce Output Operator [RS_42]
+                                             |     key expressions:_col1 (type: string)
+                                             |     Map-reduce partition columns:_col1 (type: string)
                                              |     sort order:+
-                                             |     Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                             |     Select Operator [SEL_13]
-                                             |        outputColumnNames:["_col0"]
-                                             |        Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                             |     Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
+                                             |     Select Operator [SEL_16]
+                                             |        outputColumnNames:["_col1"]
+                                             |        Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
                                              |        Filter Operator [FIL_101]
-                                             |           predicate:((value = 'd1value') and key is not null) (type: boolean)
-                                             |           Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                                             |           TableScan [TS_11]
-                                             |              alias:d1
-                                             |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                             |           predicate:((key = 'src1key') and value is not null) (type: boolean)
+                                             |           Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
+                                             |           TableScan [TS_14]
+                                             |              alias:src1
+                                             |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                                              |<-Map Join Operator [MAPJOIN_106]
                                                 |  condition map:[{"":"Inner Join 0 to 1"}]
-                                                |  keys:{"Map 3":"_col1 (type: string)","Map 6":"_col3 (type: string)"}
+                                                |  keys:{"Map 3":"_col2 (type: string)","Map 7":"_col0 (type: string)"}
                                                 |  outputColumnNames:["_col2","_col3","_col4","_col6"]
-                                                |  Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                                                |<-Map 6 [BROADCAST_EDGE]
-                                                |  Reduce Output Operator [RS_32]
-                                                |     key expressions:_col3 (type: string)
-                                                |     Map-reduce partition columns:_col3 (type: string)
+                                                |  Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+                                                |<-Map 7 [BROADCAST_EDGE]
+                                                |  Reduce Output Operator [RS_37]
+                                                |     key expressions:_col0 (type: string)
+                                                |     Map-reduce partition columns:_col0 (type: string)
                                                 |     sort order:+
-                                                |     Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                                |     value expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string), _col4 (type: string)
-                                                |     Select Operator [SEL_10]
-                                                |        outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
-                                                |        Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                                |     Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                                |     Select Operator [SEL_13]
+                                                |        outputColumnNames:["_col0"]
+                                                |        Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
                                                 |        Filter Operator [FIL_100]
-                                                |           predicate:((((((v3 = 'ssv3') and v2 is not null) and k1 is not null) and v1 is not null) and k2 is not null) and k3 is not null) (type: boolean)
-                                                |           Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
-                                                |           TableScan [TS_8]
-                                                |              alias:ss
-                                                |              Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
-                                                |<-Select Operator [SEL_7]
-                                                      outputColumnNames:["_col1"]
-                                                      Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                                      Filter Operator [FIL_99]
-                                                         predicate:((key = 'srcpartkey') and value is not null) (type: boolean)
+                                                |           predicate:((value = 'd1value') and key is not null) (type: boolean)
+                                                |           Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                                                |           TableScan [TS_11]
+                                                |              alias:d1
+                                                |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                                |<-Map Join Operator [MAPJOIN_105]
+                                                   |  condition map:[{"":"Inner Join 0 to 1"}]
+                                                   |  keys:{"Map 3":"_col1 (type: string)","Map 6":"_col3 (type: string)"}
+                                                   |  outputColumnNames:["_col2","_col3","_col4","_col6"]
+                                                   |  Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                                                   |<-Map 6 [BROADCAST_EDGE]
+                                                   |  Reduce Output Operator [RS_32]
+                                                   |     key expressions:_col3 (type: string)
+                                                   |     Map-reduce partition columns:_col3 (type: string)
+                                                   |     sort order:+
+                                                   |     Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                                   |     value expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string), _col4 (type: string)
+                                                   |     Select Operator [SEL_10]
+                                                   |        outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
+                                                   |        Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                                   |        Filter Operator [FIL_99]
+                                                   |           predicate:((((((v3 = 'ssv3') and v2 is not null) and k1 is not null) and v1 is not null) and k2 is not null) and k3 is not null) (type: boolean)
+                                                   |           Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+                                                   |           TableScan [TS_8]
+                                                   |              alias:ss
+                                                   |              Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
+                                                   |<-Select Operator [SEL_7]
+                                                         outputColumnNames:["_col1"]
                                                          Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                                         TableScan [TS_5]
-                                                            alias:srcpart
-                                                            Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
+                                                         Filter Operator [FIL_98]
+                                                            predicate:((key = 'srcpartkey') and value is not null) (type: boolean)
+                                                            Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                                            TableScan [TS_5]
+                                                               alias:srcpart
+                                                               Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
 
 PREHOOK: query: explain
 SELECT x.key, z.value, y.value


[13/50] [abbrv] hive git commit: HIVE-11781 : Remove HiveLimit operator (Jesus Camacho Rodriguez via Ashutosh Chauhan)

Posted by xu...@apache.org.
HIVE-11781 : Remove HiveLimit operator (Jesus Camacho Rodriguez via Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b1fffd5a
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b1fffd5a
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b1fffd5a

Branch: refs/heads/spark
Commit: b1fffd5a8a8aa10f19c03f79cf02d8147222ccc8
Parents: 753fed6
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Thu Sep 10 02:46:00 2015 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Sep 11 17:12:21 2015 -0700

----------------------------------------------------------------------
 .../calcite/reloperators/HiveLimit.java         |  57 ----------
 .../calcite/reloperators/HiveSort.java          | 110 -------------------
 .../calcite/reloperators/HiveSortLimit.java     | 110 +++++++++++++++++++
 .../calcite/stats/HiveRelMdMemory.java          |   9 +-
 .../calcite/stats/HiveRelMdParallelism.java     |   4 +-
 .../calcite/translator/ASTConverter.java        |  24 ++--
 .../calcite/translator/HiveOpConverter.java     |   8 +-
 .../translator/PlanModifierForASTConv.java      |  10 +-
 .../calcite/translator/PlanModifierUtil.java    |   4 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  10 +-
 10 files changed, 142 insertions(+), 204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b1fffd5a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveLimit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveLimit.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveLimit.java
deleted file mode 100644
index 74991d6..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveLimit.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * 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.optimizer.calcite.reloperators;
-
-import java.util.List;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.SingleRel;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rex.RexNode;
-import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
-
-public class HiveLimit extends SingleRel implements HiveRelNode {
-  private final RexNode offset;
-  private final RexNode fetch;
-
-  HiveLimit(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode offset,
-      RexNode fetch) {
-    super(cluster, TraitsUtil.getDefaultTraitSet(cluster), child);
-    this.offset = offset;
-    this.fetch = fetch;
-    assert getConvention() instanceof HiveRelNode;
-    assert getConvention() == child.getConvention();
-  }
-
-  @Override
-  public HiveLimit copy(RelTraitSet traitSet, List<RelNode> newInputs) {
-    return new HiveLimit(getCluster(), traitSet, sole(newInputs), offset, fetch);
-  }
-
-  public void implement(Implementor implementor) {
-  }
-
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return RelMetadataQuery.getNonCumulativeCost(this);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b1fffd5a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSort.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSort.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSort.java
deleted file mode 100644
index 1df6542..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSort.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/**
- * 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.optimizer.calcite.reloperators;
-
-import java.util.Map;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rex.RexNode;
-import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
-
-import com.google.common.collect.ImmutableMap;
-
-public class HiveSort extends Sort implements HiveRelNode {
-
-  public static final HiveSortRelFactory HIVE_SORT_REL_FACTORY = new HiveSortRelFactory();
-
-  // NOTE: this is to work around Hive Calcite Limitations w.r.t OB.
-  // 1. Calcite can not accept expressions in OB; instead it needs to be expressed
-  // as VC in input Select.
-  // 2. Hive can not preserve ordering through select boundaries.
-  // 3. This map is used for outermost OB to migrate the VC corresponding OB
-  // expressions from input select.
-  // 4. This is used by ASTConverter after we are done with Calcite Planning
-  private ImmutableMap<Integer, RexNode> mapOfInputRefToRexCall;
-
-  public HiveSort(RelOptCluster cluster, RelTraitSet traitSet, RelNode child,
-      RelCollation collation, RexNode offset, RexNode fetch) {
-    super(cluster, TraitsUtil.getSortTraitSet(cluster, traitSet, collation), child, collation,
-        offset, fetch);
-  }
-
-  /**
-   * Creates a HiveSort.
-   *
-   * @param input     Input relational expression
-   * @param collation array of sort specifications
-   * @param offset    Expression for number of rows to discard before returning
-   *                  first row
-   * @param fetch     Expression for number of rows to fetch
-   */
-  public static HiveSort create(RelNode input, RelCollation collation,
-      RexNode offset, RexNode fetch) {
-    RelOptCluster cluster = input.getCluster();
-    collation = RelCollationTraitDef.INSTANCE.canonize(collation);
-    RelTraitSet traitSet =
-        TraitsUtil.getSortTraitSet(cluster, input.getTraitSet(), collation);
-    return new HiveSort(cluster, traitSet, input, collation, offset, fetch);
-  }
-
-  @Override
-  public HiveSort copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation,
-      RexNode offset, RexNode fetch) {
-    // TODO: can we blindly copy sort trait? What if inputs changed and we
-    // are now sorting by different cols
-    RelCollation canonizedCollation = traitSet.canonize(newCollation);
-    return new HiveSort(getCluster(), traitSet, newInput, canonizedCollation, offset, fetch);
-  }
-
-  public RexNode getFetchExpr() {
-    return fetch;
-  }
-
-  public void setInputRefToCallMap(ImmutableMap<Integer, RexNode> refToCall) {
-    this.mapOfInputRefToRexCall = refToCall;
-  }
-
-  public Map<Integer, RexNode> getInputRefToCallMap() {
-    return this.mapOfInputRefToRexCall;
-  }
-
-  @Override
-  public void implement(Implementor implementor) {
-  }
-
-  private static class HiveSortRelFactory implements RelFactories.SortFactory {
-
-    @Override
-    public RelNode createSort(RelTraitSet traits, RelNode input, RelCollation collation,
-        RexNode offset, RexNode fetch) {
-      return createSort(input, collation, offset, fetch);
-    }
-
-    @Override
-    public RelNode createSort(RelNode input, RelCollation collation, RexNode offset,
-        RexNode fetch) {
-      return create(input, collation, offset, fetch);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b1fffd5a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortLimit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortLimit.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortLimit.java
new file mode 100644
index 0000000..82fdc0e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortLimit.java
@@ -0,0 +1,110 @@
+/**
+ * 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.optimizer.calcite.reloperators;
+
+import java.util.Map;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexNode;
+import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
+
+import com.google.common.collect.ImmutableMap;
+
+public class HiveSortLimit extends Sort implements HiveRelNode {
+
+  public static final HiveSortRelFactory HIVE_SORT_REL_FACTORY = new HiveSortRelFactory();
+
+  // NOTE: this is to work around Hive Calcite Limitations w.r.t OB.
+  // 1. Calcite can not accept expressions in OB; instead it needs to be expressed
+  // as VC in input Select.
+  // 2. Hive can not preserve ordering through select boundaries.
+  // 3. This map is used for outermost OB to migrate the VC corresponding OB
+  // expressions from input select.
+  // 4. This is used by ASTConverter after we are done with Calcite Planning
+  private ImmutableMap<Integer, RexNode> mapOfInputRefToRexCall;
+
+  public HiveSortLimit(RelOptCluster cluster, RelTraitSet traitSet, RelNode child,
+      RelCollation collation, RexNode offset, RexNode fetch) {
+    super(cluster, TraitsUtil.getSortTraitSet(cluster, traitSet, collation), child, collation,
+        offset, fetch);
+  }
+
+  /**
+   * Creates a HiveSortLimit.
+   *
+   * @param input     Input relational expression
+   * @param collation array of sort specifications
+   * @param offset    Expression for number of rows to discard before returning
+   *                  first row
+   * @param fetch     Expression for number of rows to fetch
+   */
+  public static HiveSortLimit create(RelNode input, RelCollation collation,
+      RexNode offset, RexNode fetch) {
+    RelOptCluster cluster = input.getCluster();
+    collation = RelCollationTraitDef.INSTANCE.canonize(collation);
+    RelTraitSet traitSet =
+        TraitsUtil.getSortTraitSet(cluster, input.getTraitSet(), collation);
+    return new HiveSortLimit(cluster, traitSet, input, collation, offset, fetch);
+  }
+
+  @Override
+  public HiveSortLimit copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation,
+      RexNode offset, RexNode fetch) {
+    // TODO: can we blindly copy sort trait? What if inputs changed and we
+    // are now sorting by different cols
+    RelCollation canonizedCollation = traitSet.canonize(newCollation);
+    return new HiveSortLimit(getCluster(), traitSet, newInput, canonizedCollation, offset, fetch);
+  }
+
+  public RexNode getFetchExpr() {
+    return fetch;
+  }
+
+  public void setInputRefToCallMap(ImmutableMap<Integer, RexNode> refToCall) {
+    this.mapOfInputRefToRexCall = refToCall;
+  }
+
+  public Map<Integer, RexNode> getInputRefToCallMap() {
+    return this.mapOfInputRefToRexCall;
+  }
+
+  @Override
+  public void implement(Implementor implementor) {
+  }
+
+  private static class HiveSortRelFactory implements RelFactories.SortFactory {
+
+    @Override
+    public RelNode createSort(RelTraitSet traits, RelNode input, RelCollation collation,
+        RexNode offset, RexNode fetch) {
+      return createSort(input, collation, offset, fetch);
+    }
+
+    @Override
+    public RelNode createSort(RelNode input, RelCollation collation, RexNode offset,
+        RexNode fetch) {
+      return create(input, collation, offset, fetch);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/b1fffd5a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdMemory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdMemory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdMemory.java
index 1a2e6d1..bea5943 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdMemory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdMemory.java
@@ -26,9 +26,8 @@ import org.apache.calcite.util.BuiltInMethod;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSort;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion;
 
@@ -73,15 +72,11 @@ public class HiveRelMdMemory extends RelMdMemory {
     return join.getCumulativeMemoryWithinPhaseSplit();
   }
 
-  public Double memory(HiveLimit limit) {
-    return 0.0;
-  }
-
   public Double memory(HiveProject project) {
     return 0.0;
   }
 
-  public Double memory(HiveSort sort) {
+  public Double memory(HiveSortLimit sort) {
     if (sort.getCollation() != RelCollations.EMPTY) {
       // It sorts
       final Double avgRowSize = RelMetadataQuery.getAverageRowSize(sort.getInput());

http://git-wip-us.apache.org/repos/asf/hive/blob/b1fffd5a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdParallelism.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdParallelism.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdParallelism.java
index c7afea9..2f51d3b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdParallelism.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdParallelism.java
@@ -27,7 +27,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSort;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 
 public class HiveRelMdParallelism extends RelMdParallelism {
@@ -52,7 +52,7 @@ public class HiveRelMdParallelism extends RelMdParallelism {
     return join.isPhaseTransition();
   }
 
-  public Boolean isPhaseTransition(HiveSort sort) {
+  public Boolean isPhaseTransition(HiveSortLimit sort) {
     // As Exchange operator is introduced later on, we make a
     // sort operator create a new stage for the moment
     return true;

http://git-wip-us.apache.org/repos/asf/hive/blob/b1fffd5a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
index b6995c9..14946b3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
@@ -58,7 +58,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSort;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.SqlFunctionConverter.HiveToken;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
@@ -204,17 +204,17 @@ public class ASTConverter {
      * to its src/from. Hence the need to pass in sort for each block from
      * its parent.
      */
-    convertOBToASTNode((HiveSort) order);
+    convertOBToASTNode((HiveSortLimit) order);
 
     // 8. Limit
-    convertLimitToASTNode((HiveSort) limit);
+    convertLimitToASTNode((HiveSortLimit) limit);
 
     return hiveAST.getAST();
   }
 
-  private void convertLimitToASTNode(HiveSort limit) {
+  private void convertLimitToASTNode(HiveSortLimit limit) {
     if (limit != null) {
-      HiveSort hiveLimit = limit;
+      HiveSortLimit hiveLimit = limit;
       RexNode limitExpr = hiveLimit.getFetchExpr();
       if (limitExpr != null) {
         Object val = ((RexLiteral) limitExpr).getValue2();
@@ -223,18 +223,18 @@ public class ASTConverter {
     }
   }
 
-  private void convertOBToASTNode(HiveSort order) {
+  private void convertOBToASTNode(HiveSortLimit order) {
     if (order != null) {
-      HiveSort hiveSort = order;
-      if (!hiveSort.getCollation().getFieldCollations().isEmpty()) {
+      HiveSortLimit hiveSortLimit = order;
+      if (!hiveSortLimit.getCollation().getFieldCollations().isEmpty()) {
         // 1 Add order by token
         ASTNode orderAst = ASTBuilder.createAST(HiveParser.TOK_ORDERBY, "TOK_ORDERBY");
 
-        schema = new Schema(hiveSort);
-        Map<Integer, RexNode> obRefToCallMap = hiveSort.getInputRefToCallMap();
+        schema = new Schema(hiveSortLimit);
+        Map<Integer, RexNode> obRefToCallMap = hiveSortLimit.getInputRefToCallMap();
         RexNode obExpr;
         ASTNode astCol;
-        for (RelFieldCollation c : hiveSort.getCollation().getFieldCollations()) {
+        for (RelFieldCollation c : hiveSortLimit.getCollation().getFieldCollations()) {
 
           // 2 Add Direction token
           ASTNode directionAST = c.getDirection() == RelFieldCollation.Direction.ASCENDING ? ASTBuilder
@@ -651,7 +651,7 @@ public class ASTConverter {
      *          Hive Sort Node
      * @return Schema
      */
-    public Schema(HiveSort order) {
+    public Schema(HiveSortLimit order) {
       Project select = (Project) order.getInput();
       for (String projName : select.getRowType().getFieldNames()) {
         add(new ColumnInfo(null, projName));

http://git-wip-us.apache.org/repos/asf/hive/blob/b1fffd5a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
index 3f66893..f6c0114 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
@@ -64,7 +64,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSort;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion;
@@ -171,8 +171,8 @@ public class HiveOpConverter {
       return visit(hj);
     } else if (rn instanceof HiveFilter) {
       return visit((HiveFilter) rn);
-    } else if (rn instanceof HiveSort) {
-      return visit((HiveSort) rn);
+    } else if (rn instanceof HiveSortLimit) {
+      return visit((HiveSortLimit) rn);
     } else if (rn instanceof HiveUnion) {
       return visit((HiveUnion) rn);
     } else if (rn instanceof HiveSortExchange) {
@@ -398,7 +398,7 @@ public class HiveOpConverter {
     return HiveGBOpConvUtil.translateGB(inputOpAf, aggRel, hiveConf);
   }
 
-  OpAttr visit(HiveSort sortRel) throws SemanticException {
+  OpAttr visit(HiveSortLimit sortRel) throws SemanticException {
     OpAttr inputOpAf = dispatch(sortRel.getInput());
 
     if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b1fffd5a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
index 5cd3a06..67f17c2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSort;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 
@@ -141,12 +141,12 @@ public class PlanModifierForASTConv {
         if (!validFilterParent(rel, parent)) {
           introduceDerivedTable(rel, parent);
         }
-      } else if (rel instanceof HiveSort) {
+      } else if (rel instanceof HiveSortLimit) {
         if (!validSortParent(rel, parent)) {
           introduceDerivedTable(rel, parent);
         }
-        if (!validSortChild((HiveSort) rel)) {
-          introduceDerivedTable(((HiveSort) rel).getInput(), rel);
+        if (!validSortChild((HiveSortLimit) rel)) {
+          introduceDerivedTable(((HiveSortLimit) rel).getInput(), rel);
         }
       } else if (rel instanceof HiveAggregate) {
         RelNode newParent = parent;
@@ -297,7 +297,7 @@ public class PlanModifierForASTConv {
     return validParent;
   }
 
-  private static boolean validSortChild(HiveSort sortNode) {
+  private static boolean validSortChild(HiveSortLimit sortNode) {
     boolean validChild = true;
     RelNode child = sortNode.getInput();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b1fffd5a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierUtil.java
index 3e2fae9..988d6d3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierUtil.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSort;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 
 import com.google.common.collect.ImmutableMap;
 
@@ -53,7 +53,7 @@ public class PlanModifierUtil {
         || !HiveCalciteUtil.orderRelNode(topSelparentPair.getKey())) {
       return;
     }
-    HiveSort obRel = (HiveSort) topSelparentPair.getKey();
+    HiveSortLimit obRel = (HiveSortLimit) topSelparentPair.getKey();
     Project obChild = (Project) topSelparentPair.getValue();
     if (obChild.getRowType().getFieldCount() <= resultSchema.size()) {
       return;

http://git-wip-us.apache.org/repos/asf/hive/blob/b1fffd5a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 73ae7c4..86bdf7e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -131,7 +131,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveRelNode;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSort;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveExpandDistinctAggregatesRule;
@@ -922,7 +922,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         // thus we run the field trimmer again to push them back down
         HiveRelFieldTrimmer fieldTrimmer = new HiveRelFieldTrimmer(null, HiveProject.DEFAULT_PROJECT_FACTORY,
             HiveFilter.DEFAULT_FILTER_FACTORY, HiveJoin.HIVE_JOIN_FACTORY,
-            HiveSemiJoin.HIVE_SEMIJOIN_FACTORY, HiveSort.HIVE_SORT_REL_FACTORY,
+            HiveSemiJoin.HIVE_SEMIJOIN_FACTORY, HiveSortLimit.HIVE_SORT_REL_FACTORY,
             HiveAggregate.HIVE_AGGR_REL_FACTORY, HiveUnion.UNION_REL_FACTORY);
         calciteOptimizedPlan = fieldTrimmer.trim(calciteOptimizedPlan);
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(),
@@ -1003,7 +1003,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       // 5. Projection Pruning
       HiveRelFieldTrimmer fieldTrimmer = new HiveRelFieldTrimmer(null, HiveProject.DEFAULT_PROJECT_FACTORY,
           HiveFilter.DEFAULT_FILTER_FACTORY, HiveJoin.HIVE_JOIN_FACTORY,
-          HiveSemiJoin.HIVE_SEMIJOIN_FACTORY, HiveSort.HIVE_SORT_REL_FACTORY,
+          HiveSemiJoin.HIVE_SEMIJOIN_FACTORY, HiveSortLimit.HIVE_SORT_REL_FACTORY,
           HiveAggregate.HIVE_AGGR_REL_FACTORY, HiveUnion.UNION_REL_FACTORY);
       basePlan = fieldTrimmer.trim(basePlan);
 
@@ -2266,7 +2266,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         // 4. Construct SortRel
         RelTraitSet traitSet = cluster.traitSetOf(HiveRelNode.CONVENTION);
         RelCollation canonizedCollation = traitSet.canonize(RelCollationImpl.of(fieldCollations));
-        sortRel = new HiveSort(cluster, traitSet, obInputRel, canonizedCollation, null, null);
+        sortRel = new HiveSortLimit(cluster, traitSet, obInputRel, canonizedCollation, null, null);
 
         // 5. Update the maps
         // NOTE: Output RR for SortRel is considered same as its input; we may
@@ -2292,7 +2292,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         RexNode fetch = cluster.getRexBuilder().makeExactLiteral(BigDecimal.valueOf(limit));
         RelTraitSet traitSet = cluster.traitSetOf(HiveRelNode.CONVENTION);
         RelCollation canonizedCollation = traitSet.canonize(RelCollations.EMPTY);
-        sortRel = new HiveSort(cluster, traitSet, srcRel, canonizedCollation, null, fetch);
+        sortRel = new HiveSortLimit(cluster, traitSet, srcRel, canonizedCollation, null, fetch);
 
         RowResolver outputRR = new RowResolver();
         if (!RowResolver.add(outputRR, relToHiveRR.get(srcRel))) {


[42/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/subquery_in_having.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_in_having.q.out b/ql/src/test/results/clientpositive/subquery_in_having.q.out
index da1da06..deab017 100644
--- a/ql/src/test/results/clientpositive/subquery_in_having.q.out
+++ b/ql/src/test/results/clientpositive/subquery_in_having.q.out
@@ -73,11 +73,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -146,22 +146,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (key > '9') (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -287,22 +283,18 @@ STAGE PLANS:
             Filter Operator
               predicate: value is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string), value (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col2 (type: bigint)
+                  value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -362,22 +354,18 @@ STAGE PLANS:
             Filter Operator
               predicate: ((key > '9') and value is not null) (type: boolean)
               Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: value (type: string), key (type: string)
-                outputColumnNames: _col0, _col1
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string), value (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col2 (type: bigint)
+                  value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -386,14 +374,14 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: string), _col2 (type: bigint)
-            outputColumnNames: _col0, _col2
+            expressions: _col1 (type: string), _col2 (type: bigint)
+            outputColumnNames: _col1, _col2
             Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: _col2 is not null (type: boolean)
               Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: _col2 (type: bigint), _col0 (type: string)
+                expressions: _col2 (type: bigint), _col1 (type: string)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
@@ -454,22 +442,18 @@ STAGE PLANS:
             Filter Operator
               predicate: p_mfgr is not null (type: boolean)
               Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: p_mfgr (type: string), p_size (type: int)
+              Group By Operator
+                aggregations: avg(p_size)
+                keys: p_mfgr (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: avg(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: struct<count:bigint,sum:double,input:int>)
+                  value expressions: _col1 (type: struct<count:bigint,sum:double,input:int>)
       Reduce Operator Tree:
         Group By Operator
           aggregations: avg(VALUE._col0)
@@ -526,22 +510,18 @@ STAGE PLANS:
             Filter Operator
               predicate: p_mfgr is not null (type: boolean)
               Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: p_mfgr (type: string), p_size (type: int)
-                outputColumnNames: _col0, _col1
+              Group By Operator
+                aggregations: max(p_size), min(p_size)
+                keys: p_mfgr (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: max(_col1), min(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: int), _col2 (type: int)
+                  value expressions: _col1 (type: int), _col2 (type: int)
       Reduce Operator Tree:
         Group By Operator
           aggregations: max(VALUE._col0), min(VALUE._col1)
@@ -612,44 +592,36 @@ STAGE PLANS:
             Filter Operator
               predicate: p_mfgr is not null (type: boolean)
               Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: p_mfgr (type: string), p_size (type: int)
+              Group By Operator
+                aggregations: avg(p_size)
+                keys: p_mfgr (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: avg(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: struct<count:bigint,sum:double,input:int>)
+                  value expressions: _col1 (type: struct<count:bigint,sum:double,input:int>)
           TableScan
             alias: b
             Statistics: Num rows: 30 Data size: 3173 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: p_mfgr is not null (type: boolean)
               Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: p_mfgr (type: string), p_size (type: int)
-                outputColumnNames: _col0, _col1
+              Group By Operator
+                aggregations: max(p_size), min(p_size)
+                keys: p_mfgr (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: max(_col1), min(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 15 Data size: 1586 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: int), _col2 (type: int)
+                  value expressions: _col1 (type: int), _col2 (type: int)
       Reduce Operator Tree:
         Demux Operator
           Statistics: Num rows: 30 Data size: 3172 Basic stats: COMPLETE Column stats: NONE
@@ -905,22 +877,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (key > '9') (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -1005,22 +973,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (key > '9') (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -1244,22 +1208,18 @@ STAGE PLANS:
             Filter Operator
               predicate: p_name is not null (type: boolean)
               Statistics: Num rows: 8 Data size: 1692 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: p_mfgr (type: string), p_name (type: string), p_size (type: int)
+              Group By Operator
+                aggregations: avg(p_size)
+                keys: p_name (type: string), p_mfgr (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 8 Data size: 1692 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: avg(_col2)
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 8 Data size: 1692 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 8 Data size: 1692 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col2 (type: struct<count:bigint,sum:double,input:int>)
+                  value expressions: _col2 (type: struct<count:bigint,sum:double,input:int>)
       Reduce Operator Tree:
         Group By Operator
           aggregations: avg(VALUE._col0)
@@ -1267,12 +1227,16 @@ STAGE PLANS:
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 4 Data size: 846 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            table:
-                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+          Select Operator
+            expressions: _col1 (type: string), _col0 (type: string), _col2 (type: double)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 4 Data size: 846 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-5
     Conditional Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/subquery_notexists.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_notexists.q.out b/ql/src/test/results/clientpositive/subquery_notexists.q.out
index 81b4137..215d855 100644
--- a/ql/src/test/results/clientpositive/subquery_notexists.q.out
+++ b/ql/src/test/results/clientpositive/subquery_notexists.q.out
@@ -257,20 +257,16 @@ STAGE PLANS:
             Filter Operator
               predicate: (value > 'val_2') (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                keys: key (type: string), value (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/subquery_notexists_having.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_notexists_having.q.out b/ql/src/test/results/clientpositive/subquery_notexists_having.q.out
index fd09901..637fc62 100644
--- a/ql/src/test/results/clientpositive/subquery_notexists_having.q.out
+++ b/ql/src/test/results/clientpositive/subquery_notexists_having.q.out
@@ -34,10 +34,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -186,10 +186,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -260,20 +260,16 @@ STAGE PLANS:
             Filter Operator
               predicate: (value > 'val_12') (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                keys: key (type: string), value (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/subquery_notin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_notin.q.out b/ql/src/test/results/clientpositive/subquery_notin.q.out
index fd6d53b..5563794 100644
--- a/ql/src/test/results/clientpositive/subquery_notin.q.out
+++ b/ql/src/test/results/clientpositive/subquery_notin.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: -- non agg, non corr
 explain
 select * 
@@ -151,7 +151,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: select * 
 from src 
 where src.key not in  ( select key from src s1 where s1.key > '2')
@@ -285,7 +285,7 @@ POSTHOOK: Input: default@src
 199	val_199
 199	val_199
 2	val_2
-Warning: Shuffle Join JOIN[31][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: -- non agg, corr
 explain
 select p_mfgr, b.p_name, p_size 
@@ -528,7 +528,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[31][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: select p_mfgr, b.p_name, p_size 
 from part b 
 where b.p_name not in 
@@ -567,7 +567,7 @@ Manufacturer#4	almond azure aquamarine papaya violet	12
 Manufacturer#5	almond antique blue firebrick mint	31
 Manufacturer#5	almond aquamarine dodger light gainsboro	46
 Manufacturer#5	almond azure blanched chiffon midnight	23
-Warning: Shuffle Join JOIN[43][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: -- agg, non corr
 explain
 select p_name, p_size 
@@ -843,7 +843,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[43][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: select p_name, p_size 
 from 
 part where part.p_size not in 
@@ -890,7 +890,7 @@ almond aquamarine sandy cyan gainsboro	18
 almond aquamarine yellow dodger mint	7
 almond azure aquamarine papaya violet	12
 almond azure blanched chiffon midnight	23
-Warning: Shuffle Join JOIN[40][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: -- agg, corr
 explain
 select p_mfgr, p_name, p_size 
@@ -1202,7 +1202,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[40][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: select p_mfgr, p_name, p_size 
 from part b where b.p_size not in 
   (select min(p_size) 
@@ -1243,7 +1243,7 @@ Manufacturer#5	almond antique medium spring khaki	6
 Manufacturer#5	almond azure blanched chiffon midnight	23
 Manufacturer#5	almond antique blue firebrick mint	31
 Manufacturer#5	almond aquamarine dodger light gainsboro	46
-Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: -- non agg, non corr, Group By in Parent Query
 select li.l_partkey, count(*) 
 from lineitem li 
@@ -1278,7 +1278,7 @@ POSTHOOK: Input: default@lineitem
 139636	1
 175839	1
 182052	1
-Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: -- alternate not in syntax
 select * 
 from src 
@@ -1442,7 +1442,7 @@ POSTHOOK: Input: default@src
 POSTHOOK: Input: default@t1_v
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@T2_v
-Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: explain
 select * 
 from T1_v where T1_v.key not in (select T2_v.key from T2_v)
@@ -1587,7 +1587,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: select * 
 from T1_v where T1_v.key not in (select T2_v.key from T2_v)
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/subquery_notin_having.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_notin_having.q.java1.7.out b/ql/src/test/results/clientpositive/subquery_notin_having.q.java1.7.out
index 775f477..9689ae3 100644
--- a/ql/src/test/results/clientpositive/subquery_notin_having.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/subquery_notin_having.q.java1.7.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: -- non agg, non corr
 -- JAVA_VERSION_SPECIFIC_OUTPUT
 
@@ -39,11 +39,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -188,7 +188,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[36][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[32][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: -- non agg, corr
 explain
 select b.p_mfgr, min(p_retailprice) 
@@ -229,11 +229,11 @@ STAGE PLANS:
             Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: p_mfgr (type: string), p_retailprice (type: double)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: p_mfgr, p_retailprice
               Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: min(_col1)
-                keys: _col0 (type: string)
+                aggregations: min(p_retailprice)
+                keys: p_mfgr (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
@@ -332,11 +332,11 @@ STAGE PLANS:
             Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: p_mfgr (type: string), p_retailprice (type: double)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: p_mfgr, p_retailprice
               Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: min(_col1), max(_col1)
-                keys: _col0 (type: string)
+                aggregations: min(p_retailprice), max(p_retailprice)
+                keys: p_mfgr (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
@@ -404,11 +404,11 @@ STAGE PLANS:
             Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: p_mfgr (type: string), p_retailprice (type: double)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: p_mfgr, p_retailprice
               Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: min(_col1), max(_col1)
-                keys: _col0 (type: string)
+                aggregations: min(p_retailprice), max(p_retailprice)
+                keys: p_mfgr (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
@@ -445,7 +445,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[36][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[32][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: select b.p_mfgr, min(p_retailprice) 
 from part b 
 group by b.p_mfgr
@@ -470,7 +470,7 @@ POSTHOOK: Input: default@part
 #### A masked pattern was here ####
 Manufacturer#1	1173.15
 Manufacturer#2	1690.68
-Warning: Shuffle Join JOIN[39][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-3:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[35][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-3:MAPRED' is a cross product
 PREHOOK: query: -- agg, non corr
 explain
 select b.p_mfgr, min(p_retailprice) 
@@ -513,11 +513,11 @@ STAGE PLANS:
             Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: p_mfgr (type: string), p_retailprice (type: double)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: p_mfgr, p_retailprice
               Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: min(_col1)
-                keys: _col0 (type: string)
+                aggregations: min(p_retailprice)
+                keys: p_mfgr (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
@@ -613,11 +613,11 @@ STAGE PLANS:
             Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: p_mfgr (type: string), p_retailprice (type: double)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: p_mfgr, p_retailprice
               Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: max(_col1), min(_col1)
-                keys: _col0 (type: string)
+                aggregations: max(p_retailprice), min(p_retailprice)
+                keys: p_mfgr (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
@@ -659,11 +659,11 @@ STAGE PLANS:
               Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: null (type: string), p_retailprice (type: double)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: p_mfgr, p_retailprice
                 Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: max(_col1), min(_col1)
-                  keys: _col0 (type: string)
+                  aggregations: max(p_retailprice), min(p_retailprice)
+                  keys: p_mfgr (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
@@ -733,7 +733,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[39][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-3:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[35][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-3:MAPRED' is a cross product
 PREHOOK: query: select b.p_mfgr, min(p_retailprice) 
 from part b 
 group by b.p_mfgr

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out b/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
index 76d7503..908ad39 100644
--- a/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
+++ b/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
@@ -549,20 +549,16 @@ STAGE PLANS:
             Filter Operator
               predicate: ((key > '9') and value is not null) (type: boolean)
               Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                keys: key (type: string), value (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
@@ -658,22 +654,18 @@ STAGE PLANS:
             Filter Operator
               predicate: value is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string), value (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col2 (type: bigint)
+                  value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -733,22 +725,18 @@ STAGE PLANS:
             Filter Operator
               predicate: ((key > '9') and value is not null) (type: boolean)
               Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: value (type: string), key (type: string)
-                outputColumnNames: _col0, _col1
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string), value (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col2 (type: bigint)
+                  value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -757,14 +745,14 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: string), _col2 (type: bigint)
-            outputColumnNames: _col0, _col2
+            expressions: _col1 (type: string), _col2 (type: bigint)
+            outputColumnNames: _col1, _col2
             Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: _col2 is not null (type: boolean)
               Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: _col2 (type: bigint), _col0 (type: string)
+                expressions: _col2 (type: bigint), _col1 (type: string)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
@@ -785,7 +773,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[31][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: -- non agg, corr
 explain
 select p_mfgr, b.p_name, p_size 

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/subquery_views.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_views.q.out b/ql/src/test/results/clientpositive/subquery_views.q.out
index c59d86e..cfa7339 100644
--- a/ql/src/test/results/clientpositive/subquery_views.q.out
+++ b/ql/src/test/results/clientpositive/subquery_views.q.out
@@ -69,8 +69,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@cv2
-Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[52][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-6:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[50][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-6:MAPRED' is a cross product
 PREHOOK: query: explain
 select * 
 from cv2 where cv2.key in (select key from cv2 c where c.key < '11')
@@ -378,8 +378,8 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[52][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-6:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[50][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-6:MAPRED' is a cross product
 PREHOOK: query: select * 
 from cv2 where cv2.key in (select key from cv2 c where c.key < '11')
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/count.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/count.q.out b/ql/src/test/results/clientpositive/tez/count.q.out
index a5e6082..b474f05 100644
--- a/ql/src/test/results/clientpositive/tez/count.q.out
+++ b/ql/src/test/results/clientpositive/tez/count.q.out
@@ -53,11 +53,11 @@ STAGE PLANS:
                   Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: a (type: int), b (type: int), c (type: int), d (type: int)
-                    outputColumnNames: _col0, _col1, _col2, _col3
+                    outputColumnNames: a, b, c, d
                     Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1), count(DISTINCT _col2), sum(_col3)
-                      keys: _col0 (type: int), _col1 (type: int), _col2 (type: int)
+                      aggregations: count(DISTINCT b), count(DISTINCT c), sum(d)
+                      keys: a (type: int), b (type: int), c (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                       Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
@@ -188,14 +188,14 @@ STAGE PLANS:
                   Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: a (type: int), b (type: int), c (type: int), d (type: int)
-                    outputColumnNames: _col0, _col1, _col2, _col3
+                    outputColumnNames: a, b, c, d
                     Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int)
+                      key expressions: a (type: int), b (type: int), c (type: int)
                       sort order: +++
-                      Map-reduce partition columns: _col0 (type: int)
+                      Map-reduce partition columns: a (type: int)
                       Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col3 (type: int)
+                      value expressions: d (type: int)
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
index cd2de52..e11f3e5 100644
--- a/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
@@ -59,10 +59,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string)
+                      keys: ds (type: string)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
@@ -1773,7 +1773,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 #### A masked pattern was here ####
 1000
-Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: -- parent is reduce tasks
 EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'
 PREHOOK: type: QUERY
@@ -1812,10 +1812,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: '2008-04-08' (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string)
+                      keys: ds (type: string)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -1875,7 +1875,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@srcpart
@@ -2739,10 +2739,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -2757,10 +2757,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0)
+                      aggregations: min(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -2936,10 +2936,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -2954,10 +2954,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0)
+                      aggregations: min(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -3121,20 +3121,16 @@ STAGE PLANS:
                   alias: srcpart
                   filterExpr: ds is not null (type: boolean)
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: ds (type: string)
+                  Group By Operator
+                    keys: ds (type: string)
+                    mode: hash
                     outputColumnNames: _col0
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                    Group By Operator
-                      keys: _col0 (type: string)
-                      mode: hash
-                      outputColumnNames: _col0
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
         Map 10 
             Map Operator Tree:
                 TableScan
@@ -3142,10 +3138,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0)
+                      aggregations: min(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -3159,20 +3155,16 @@ STAGE PLANS:
                   alias: srcpart
                   filterExpr: ds is not null (type: boolean)
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: ds (type: string)
+                  Group By Operator
+                    keys: ds (type: string)
+                    mode: hash
                     outputColumnNames: _col0
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                    Group By Operator
-                      keys: _col0 (type: string)
-                      mode: hash
-                      outputColumnNames: _col0
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
         Map 7 
             Map Operator Tree:
                 TableScan
@@ -3180,10 +3172,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -4133,7 +4125,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 #### A masked pattern was here ####
 1000
-Warning: Map Join MAPJOIN[23][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Map 1' is a cross product
 PREHOOK: query: -- parent is reduce tasks
 EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'
 PREHOOK: type: QUERY
@@ -4188,10 +4180,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: '2008-04-08' (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string)
+                      keys: ds (type: string)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -4233,7 +4225,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Map Join MAPJOIN[23][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Map 1' is a cross product
 PREHOOK: query: select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@srcpart
@@ -4824,10 +4816,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -4842,10 +4834,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0)
+                      aggregations: min(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
index 4451046..63e6ade 100644
--- a/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
@@ -1591,36 +1591,36 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (t is null or (t = 27)) (type: boolean)
                     Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
+                    Group By Operator
+                      keys: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float)
+                      mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                        mode: hash
-                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      Reduce Output Operator
+                        key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
+                        sort order: +++++
+                        Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
                         Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                          sort order: +++++
-                          Map-reduce partition columns: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                          Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: smallint), KEY._col1 (type: int), KEY._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: smallint), KEY._col2 (type: int), KEY._col3 (type: bigint), KEY._col4 (type: float)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
+                Select Operator
+                  expressions: _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col0 (type: tinyint)
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4
                   Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part2_orc
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        name: default.over1k_part2_orc
             Execution mode: vectorized
 
   Stage: Stage-2
@@ -1659,7 +1659,6 @@ STAGE PLANS:
     Tez
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1670,49 +1669,36 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (t is null or (t = 27)) (type: boolean)
                     Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
+                    Group By Operator
+                      keys: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float)
+                      mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                        mode: hash
-                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      Reduce Output Operator
+                        key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
+                        sort order: +++++
+                        Map-reduce partition columns: _col0 (type: tinyint)
                         Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                          sort order: +++++
-                          Map-reduce partition columns: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                          Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: smallint), KEY._col1 (type: int), KEY._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: smallint), KEY._col2 (type: int), KEY._col3 (type: bigint), KEY._col4 (type: float)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col4 (type: tinyint)
-                  sort order: +
-                  Map-reduce partition columns: _col4 (type: tinyint)
+                Select Operator
+                  expressions: _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col0 (type: tinyint)
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4
                   Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-            Execution mode: vectorized
-        Reducer 3 
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
-                  Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part2_orc
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        name: default.over1k_part2_orc
             Execution mode: vectorized
 
   Stage: Stage-2

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
index cb001b9..b76b4f9 100644
--- a/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
@@ -1496,35 +1496,35 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (t is null or (t = 27)) (type: boolean)
                     Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
+                    Group By Operator
+                      keys: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float)
+                      mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                        mode: hash
-                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      Reduce Output Operator
+                        key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
+                        sort order: +++++
+                        Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
                         Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                          sort order: +++++
-                          Map-reduce partition columns: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                          Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: smallint), KEY._col1 (type: int), KEY._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: smallint), KEY._col2 (type: int), KEY._col3 (type: bigint), KEY._col4 (type: float)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
+                Select Operator
+                  expressions: _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col0 (type: tinyint)
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4
                   Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part2
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        name: default.over1k_part2
 
   Stage: Stage-2
     Dependency Collection
@@ -1562,7 +1562,6 @@ STAGE PLANS:
     Tez
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1573,47 +1572,35 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (t is null or (t = 27)) (type: boolean)
                     Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
+                    Group By Operator
+                      keys: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float)
+                      mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                        mode: hash
-                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      Reduce Output Operator
+                        key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
+                        sort order: +++++
+                        Map-reduce partition columns: _col0 (type: tinyint)
                         Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                          sort order: +++++
-                          Map-reduce partition columns: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                          Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: smallint), KEY._col1 (type: int), KEY._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: smallint), KEY._col2 (type: int), KEY._col3 (type: bigint), KEY._col4 (type: float)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col4 (type: tinyint)
-                  sort order: +
-                  Map-reduce partition columns: _col4 (type: tinyint)
-                  Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-        Reducer 3 
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
+                Select Operator
+                  expressions: _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col0 (type: tinyint)
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4
                   Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part2
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        name: default.over1k_part2
 
   Stage: Stage-2
     Dependency Collection


[25/50] [abbrv] hive git commit: HIVE-5623: ORC accessing array column that's empty will fail with java out of bound exception (Prasanth Jayachandran reviewed by Ashutosh Chauhan)

Posted by xu...@apache.org.
HIVE-5623: ORC accessing array column that's empty will fail with java out of bound exception (Prasanth Jayachandran reviewed by Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0bc96772
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0bc96772
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0bc96772

Branch: refs/heads/spark
Commit: 0bc96772e440653815792d3e37db3e9063e6c2e1
Parents: da0be3d
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Mon Sep 14 16:06:18 2015 -0500
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Mon Sep 14 16:06:18 2015 -0500

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcStruct.java     | 2 +-
 ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcStruct.java | 2 ++
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0bc96772/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcStruct.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcStruct.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcStruct.java
index b1a32bc..7a17b92 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcStruct.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcStruct.java
@@ -435,7 +435,7 @@ final public class OrcStruct implements Writable {
 
     @Override
     public Object getListElement(Object list, int i) {
-      if (list == null) {
+      if (list == null || i < 0 || i >= getListLength(list)) {
         return null;
       }
       return ((List) list).get(i);

http://git-wip-us.apache.org/repos/asf/hive/blob/0bc96772/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcStruct.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcStruct.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcStruct.java
index 8fc0693..2e431c8 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcStruct.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcStruct.java
@@ -94,6 +94,8 @@ public class TestOrcStruct {
         inspector.getAllStructFieldRefs().get(12).getFieldObjectInspector();
     assertEquals(ObjectInspector.Category.LIST, listOI.getCategory());
     assertEquals(10, listOI.getListElement(list, 10));
+    assertEquals(null, listOI.getListElement(list, -1));
+    assertEquals(null, listOI.getListElement(list, 13));
     assertEquals(13, listOI.getListLength(list));
 
     Map<Integer, Integer> map = new HashMap<Integer,Integer>();


[09/50] [abbrv] hive git commit: HIVE-11605: Incorrect results with bucket map join in tez. (Vikram Dixit K, reviewed by Sergey Shelukhin)

Posted by xu...@apache.org.
HIVE-11605: Incorrect results with bucket map join in tez. (Vikram Dixit K, reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4ea8e296
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4ea8e296
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4ea8e296

Branch: refs/heads/spark
Commit: 4ea8e29619eb0bbb02e3f7c09ffc9d44bf4cdfef
Parents: 594e25a
Author: vikram <vi...@hortonworks.com>
Authored: Thu Sep 10 13:13:56 2015 -0700
Committer: vikram <vi...@hortonworks.com>
Committed: Thu Sep 10 13:30:23 2015 -0700

----------------------------------------------------------------------
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |  18 ++-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |   8 +-
 .../clientpositive/bucket_map_join_tez1.q       |   9 ++
 .../spark/bucket_map_join_tez1.q.out            | 131 +++++++++++++++++++
 .../tez/bucket_map_join_tez1.q.out              | 123 +++++++++++++++++
 5 files changed, 280 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4ea8e296/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index e3acdfc..8ea1879 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -375,13 +375,13 @@ public class ConvertJoinMapJoin implements NodeProcessor {
       }
       ReduceSinkOperator rsOp = (ReduceSinkOperator) parentOp;
       if (checkColEquality(rsOp.getParentOperators().get(0).getOpTraits().getSortCols(), rsOp
-          .getOpTraits().getSortCols(), rsOp.getColumnExprMap(), tezBucketJoinProcCtx) == false) {
+          .getOpTraits().getSortCols(), rsOp.getColumnExprMap(), tezBucketJoinProcCtx, false) == false) {
         LOG.info("We cannot convert to SMB because the sort column names do not match.");
         return false;
       }
 
       if (checkColEquality(rsOp.getParentOperators().get(0).getOpTraits().getBucketColNames(), rsOp
-          .getOpTraits().getBucketColNames(), rsOp.getColumnExprMap(), tezBucketJoinProcCtx)
+          .getOpTraits().getBucketColNames(), rsOp.getColumnExprMap(), tezBucketJoinProcCtx, true)
           == false) {
         LOG.info("We cannot convert to SMB because bucket column names do not match.");
         return false;
@@ -428,7 +428,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     int numBuckets = parentOfParent.getOpTraits().getNumBuckets();
     // all keys matched.
     if (checkColEquality(grandParentColNames, parentColNames, rs.getColumnExprMap(),
-        tezBucketJoinProcCtx) == false) {
+        tezBucketJoinProcCtx, true) == false) {
       LOG.info("No info available to check for bucket map join. Cannot convert");
       return false;
     }
@@ -446,7 +446,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
 
   private boolean checkColEquality(List<List<String>> grandParentColNames,
       List<List<String>> parentColNames, Map<String, ExprNodeDesc> colExprMap,
-      TezBucketJoinProcCtx tezBucketJoinProcCtx) {
+      TezBucketJoinProcCtx tezBucketJoinProcCtx, boolean strict) {
 
     if ((grandParentColNames == null) || (parentColNames == null)) {
       return false;
@@ -479,7 +479,15 @@ public class ConvertJoinMapJoin implements NodeProcessor {
           }
 
           if (colCount == parentColNames.get(0).size()) {
-            return true;
+            if (strict) {
+              if (colCount == listBucketCols.size()) {
+                return true;
+              } else {
+                return false;
+              }
+            } else {
+              return true;
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ea8e296/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
index b546838..71c766f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
@@ -226,10 +226,6 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
     int numBuckets = -1;
     EdgeType edgeType = EdgeType.BROADCAST_EDGE;
     if (joinConf.isBucketMapJoin()) {
-
-      // disable auto parallelism for bucket map joins
-      parentRS.getConf().setReducerTraits(EnumSet.of(FIXED));
-
       numBuckets = (Integer) joinConf.getBigTableBucketNumMapping().values().toArray()[0];
       /*
        * Here, we can be in one of 4 states.
@@ -273,6 +269,10 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
     } else if (mapJoinOp.getConf().isDynamicPartitionHashJoin()) {
       edgeType = EdgeType.CUSTOM_SIMPLE_EDGE;
     }
+    if (edgeType == EdgeType.CUSTOM_EDGE) {
+      // disable auto parallelism for bucket map joins
+      parentRS.getConf().setReducerTraits(EnumSet.of(FIXED));
+    }
     TezEdgeProperty edgeProp = new TezEdgeProperty(null, edgeType, numBuckets);
 
     if (mapJoinWork != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/4ea8e296/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q b/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
index 4a7d63e..0f9dd6d 100644
--- a/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
+++ b/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
@@ -30,6 +30,15 @@ explain
 select a.key, a.value, b.value
 from tab a join tab_part b on a.key = b.key;
 
+explain
+select count(*)
+from 
+(select distinct key, value from tab_part) a join tab b on a.key = b.key;
+
+select count(*)
+from 
+(select distinct key, value from tab_part) a join tab b on a.key = b.key;
+
 -- one side is really bucketed. srcbucket_mapjoin is not really a bucketed table.
 -- In this case the sub-query is chosen as the big table.
 explain

http://git-wip-us.apache.org/repos/asf/hive/blob/4ea8e296/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
index 65bded2..34ddc90 100644
--- a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
@@ -183,6 +183,137 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+PREHOOK: query: explain
+select count(*)
+from 
+(select distinct key, value from tab_part) a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select count(*)
+from 
+(select distinct key, value from tab_part) a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col0 (type: int)
+                        1 key (type: int)
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+      Edges:
+        Reducer 2 <- Map 1 (GROUP, 2)
+        Reducer 3 <- Reducer 2 (GROUP, 1)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: tab_part
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: key (type: int), value (type: string)
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Local Work:
+              Map Reduce Local Work
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int), KEY._col1 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: int)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                  Map Join Operator
+                    condition map:
+                         Inner Join 0 to 1
+                    keys:
+                      0 _col0 (type: int)
+                      1 key (type: int)
+                    input vertices:
+                      1 Map 4
+                    Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*)
+from 
+(select distinct key, value from tab_part) a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tab
+PREHOOK: Input: default@tab@ds=2008-04-08
+PREHOOK: Input: default@tab_part
+PREHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*)
+from 
+(select distinct key, value from tab_part) a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tab
+POSTHOOK: Input: default@tab@ds=2008-04-08
+POSTHOOK: Input: default@tab_part
+POSTHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+242
 PREHOOK: query: -- one side is really bucketed. srcbucket_mapjoin is not really a bucketed table.
 -- In this case the sub-query is chosen as the big table.
 explain

http://git-wip-us.apache.org/repos/asf/hive/blob/4ea8e296/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
index 61c197f..8338672 100644
--- a/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
+++ b/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
@@ -178,6 +178,129 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+PREHOOK: query: explain
+select count(*)
+from 
+(select distinct key, value from tab_part) a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select count(*)
+from 
+(select distinct key, value from tab_part) a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (BROADCAST_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: tab_part
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: key (type: int), value (type: string)
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: key (type: int)
+                      sort order: +
+                      Map-reduce partition columns: key (type: int)
+                      Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int), KEY._col1 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: int)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                  Map Join Operator
+                    condition map:
+                         Inner Join 0 to 1
+                    keys:
+                      0 _col0 (type: int)
+                      1 key (type: int)
+                    input vertices:
+                      1 Map 4
+                    Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                    HybridGraceHashJoin: true
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*)
+from 
+(select distinct key, value from tab_part) a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tab
+PREHOOK: Input: default@tab@ds=2008-04-08
+PREHOOK: Input: default@tab_part
+PREHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*)
+from 
+(select distinct key, value from tab_part) a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tab
+POSTHOOK: Input: default@tab@ds=2008-04-08
+POSTHOOK: Input: default@tab_part
+POSTHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+242
 PREHOOK: query: -- one side is really bucketed. srcbucket_mapjoin is not really a bucketed table.
 -- In this case the sub-query is chosen as the big table.
 explain


[27/50] [abbrv] hive git commit: HIVE-11802: Float-point numbers are displayed with different precision in Beeline/JDBC (Sergio Pena, reviewed by Carl Steinbach)

Posted by xu...@apache.org.
HIVE-11802: Float-point numbers are displayed with different precision in Beeline/JDBC (Sergio Pena, reviewed by Carl Steinbach)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/cabd4810
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/cabd4810
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/cabd4810

Branch: refs/heads/spark
Commit: cabd4810568e00f91b8f53f8d966d9d799897dbf
Parents: 9804918
Author: Sergio Pena <se...@cloudera.com>
Authored: Tue Sep 15 10:40:45 2015 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Tue Sep 15 10:40:45 2015 -0500

----------------------------------------------------------------------
 .../org/apache/hive/service/cli/Column.java     |   3 +-
 .../org/apache/hive/service/cli/TestColumn.java | 129 +++++++++++++++++++
 2 files changed, 131 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cabd4810/service/src/java/org/apache/hive/service/cli/Column.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/Column.java b/service/src/java/org/apache/hive/service/cli/Column.java
index 2e21f18..31091a3 100644
--- a/service/src/java/org/apache/hive/service/cli/Column.java
+++ b/service/src/java/org/apache/hive/service/cli/Column.java
@@ -40,6 +40,7 @@ import org.apache.hive.service.cli.thrift.TI16Column;
 import org.apache.hive.service.cli.thrift.TI32Column;
 import org.apache.hive.service.cli.thrift.TI64Column;
 import org.apache.hive.service.cli.thrift.TStringColumn;
+import sun.misc.FloatingDecimal;
 
 /**
  * Column.
@@ -349,7 +350,7 @@ public class Column extends AbstractList {
         break;
       case FLOAT_TYPE:
         nulls.set(size, field == null);
-        doubleVars()[size] = field == null ? 0 : ((Float)field).doubleValue();
+        doubleVars()[size] = field == null ? 0 : new FloatingDecimal((Float)field).doubleValue();
         break;
       case DOUBLE_TYPE:
         nulls.set(size, field == null);

http://git-wip-us.apache.org/repos/asf/hive/blob/cabd4810/service/src/test/org/apache/hive/service/cli/TestColumn.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/TestColumn.java b/service/src/test/org/apache/hive/service/cli/TestColumn.java
new file mode 100644
index 0000000..87bf848
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/cli/TestColumn.java
@@ -0,0 +1,129 @@
+/**
+ * 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.hive.service.cli;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestColumn {
+  @Test
+  public void testAllIntegerTypeValues() {
+    Map<Type, List<Object>> integerTypesAndValues = new LinkedHashMap<Type, List<Object>>();
+
+    // Add TINYINT values
+    integerTypesAndValues.put(Type.TINYINT_TYPE, Arrays.<Object>asList(
+        Byte.MIN_VALUE, Byte.MAX_VALUE
+    ));
+
+    // Add SMALLINT values
+    integerTypesAndValues.put(Type.SMALLINT_TYPE, Arrays.<Object>asList(
+        Short.MIN_VALUE, Short.MIN_VALUE
+    ));
+
+    // Add INT values
+    integerTypesAndValues.put(Type.INT_TYPE, Arrays.<Object>asList(
+        Integer.MIN_VALUE, Integer.MAX_VALUE
+    ));
+
+    // Add BIGINT values
+    integerTypesAndValues.put(Type.BIGINT_TYPE, Arrays.<Object>asList(
+        Long.MIN_VALUE, Long.MAX_VALUE
+    ));
+
+    // Validate all integer type values are stored correctly
+    for (Map.Entry entry : integerTypesAndValues.entrySet()) {
+      Type type = (Type)entry.getKey();
+      List<Object> values = (List)entry.getValue();
+
+      Column c = new Column(type);
+      for (Object v : values) {
+        c.addValue(type, v);
+      }
+
+      assertEquals(type, c.getType());
+      assertEquals(values.size(), c.size());
+
+      for (int i=0; i<c.size(); i++) {
+        assertEquals(values.get(i), c.get(i));
+      }
+    }
+  }
+
+  @Test
+  public void testFloatAndDoubleValues() {
+    Column floatColumn = new Column(Type.FLOAT_TYPE);
+    floatColumn.addValue(Type.FLOAT_TYPE, 1.1f);
+    floatColumn.addValue(Type.FLOAT_TYPE, 2.033f);
+
+    // FLOAT_TYPE is treated as DOUBLE_TYPE
+    assertEquals(Type.DOUBLE_TYPE, floatColumn.getType());
+    assertEquals(2, floatColumn.size());
+    assertEquals(1.1, floatColumn.get(0));
+    assertEquals(2.033, floatColumn.get(1));
+
+    Column doubleColumn = new Column(Type.DOUBLE_TYPE);
+    doubleColumn.addValue(Type.DOUBLE_TYPE, 1.1);
+    doubleColumn.addValue(Type.DOUBLE_TYPE, 2.033);
+
+    assertEquals(Type.DOUBLE_TYPE, doubleColumn.getType());
+    assertEquals(2, doubleColumn.size());
+    assertEquals(1.1, doubleColumn.get(0));
+    assertEquals(2.033, doubleColumn.get(1));
+  }
+
+  @Test
+  public void testBooleanValues() {
+    Column boolColumn = new Column(Type.BOOLEAN_TYPE);
+    boolColumn.addValue(Type.BOOLEAN_TYPE, true);
+    boolColumn.addValue(Type.BOOLEAN_TYPE, false);
+
+    assertEquals(Type.BOOLEAN_TYPE, boolColumn.getType());
+    assertEquals(2, boolColumn.size());
+    assertEquals(true, boolColumn.get(0));
+    assertEquals(false, boolColumn.get(1));
+  }
+
+  @Test
+  public void testStringValues() {
+    Column stringColumn = new Column(Type.STRING_TYPE);
+    stringColumn.addValue(Type.STRING_TYPE, "12abc456");
+    stringColumn.addValue(Type.STRING_TYPE, "~special$&string");
+
+    assertEquals(Type.STRING_TYPE, stringColumn.getType());
+    assertEquals(2, stringColumn.size());
+    assertEquals("12abc456", stringColumn.get(0));
+    assertEquals("~special$&string", stringColumn.get(1));
+  }
+
+  @Test
+  public void testBinaryValues() {
+    Column binaryColumn = new Column(Type.BINARY_TYPE);
+    binaryColumn.addValue(Type.BINARY_TYPE, new byte[]{-1, 0, 3, 4});
+
+    assertEquals(Type.BINARY_TYPE, binaryColumn.getType());
+    assertEquals(1, binaryColumn.size());
+    assertArrayEquals(new byte[]{-1, 0, 3, 4}, (byte[]) binaryColumn.get(0));
+  }
+}


[23/50] [abbrv] hive git commit: HIVE-11792: User explain in tez does not preserve ordering (Prasanth Jayachandran reviewed by Pengcheng Xiong)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
index 7595c3e..36f1099 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
@@ -53,8 +53,8 @@ Stage-3
    Stats-Aggr Operator
       Stage-0
          Move Operator
-            partition:{"ts":"2012-01-03+14:46:31","ds":"2012-01-03"}
-            table:{"serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.src_orc_merge_test_part","input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"}
+            partition:{"ds":"2012-01-03","ts":"2012-01-03+14:46:31"}
+            table:{"input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat","serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.src_orc_merge_test_part"}
             Stage-2
                Dependency Collection{}
                   Stage-1
@@ -62,7 +62,7 @@ Stage-3
                      File Output Operator [FS_3]
                         compressed:false
                         Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        table:{"serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.src_orc_merge_test_part","input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"}
+                        table:{"input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat","serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.src_orc_merge_test_part"}
                         Select Operator [SEL_1]
                            outputColumnNames:["_col0","_col1"]
                            Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -93,8 +93,8 @@ Stage-3
    Stats-Aggr Operator
       Stage-0
          Move Operator
-            partition:{"ts":"2012-01-03+14:46:31","ds":"2012-01-03"}
-            table:{"serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.src_orc_merge_test_part","input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"}
+            partition:{"ds":"2012-01-03","ts":"2012-01-03+14:46:31"}
+            table:{"input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat","serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.src_orc_merge_test_part"}
             Stage-2
                Dependency Collection{}
                   Stage-1
@@ -102,7 +102,7 @@ Stage-3
                      File Output Operator [FS_7]
                         compressed:false
                         Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
-                        table:{"serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.src_orc_merge_test_part","input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"}
+                        table:{"input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat","serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.src_orc_merge_test_part"}
                         Select Operator [SEL_6]
                            outputColumnNames:["_col0","_col1"]
                            Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
@@ -144,7 +144,7 @@ Stage-0
          File Output Operator [FS_8]
             compressed:false
             Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_6]
             |  aggregations:["count(VALUE._col0)"]
             |  outputColumnNames:["_col0"]
@@ -181,7 +181,7 @@ Stage-0
          File Output Operator [FS_8]
             compressed:false
             Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_6]
             |  aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
             |  outputColumnNames:["_col0","_col1"]
@@ -227,7 +227,7 @@ Stage-0
          File Output Operator [FS_8]
             compressed:false
             Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_6]
             |  aggregations:["count(VALUE._col0)"]
             |  outputColumnNames:["_col0"]
@@ -264,7 +264,7 @@ Stage-0
          File Output Operator [FS_8]
             compressed:false
             Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_6]
             |  aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
             |  outputColumnNames:["_col0","_col1"]
@@ -329,7 +329,7 @@ Stage-0
          File Output Operator [FS_17]
             compressed:false
             Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_15]
             |  aggregations:["sum(VALUE._col0)"]
             |  outputColumnNames:["_col0"]
@@ -397,7 +397,7 @@ Stage-0
          File Output Operator [FS_7]
             compressed:false
             Statistics:Num rows: 10 Data size: 885 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_6]
                outputColumnNames:["_col0","_col1","_col2"]
                Statistics:Num rows: 10 Data size: 885 Basic stats: COMPLETE Column stats: COMPLETE
@@ -443,7 +443,7 @@ Stage-0
          File Output Operator [FS_12]
             compressed:false
             Statistics:Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_11]
                outputColumnNames:["_col0","_col1","_col2"]
                Statistics:Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE
@@ -515,7 +515,7 @@ Stage-0
          File Output Operator [FS_45]
             compressed:false
             Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_44]
             |  outputColumnNames:["_col0","_col1","_col2"]
             |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
@@ -550,7 +550,7 @@ Stage-0
                                  Statistics:Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                                  Merge Join Operator [MERGEJOIN_55]
                                  |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                 |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                  |  outputColumnNames:["_col1","_col2","_col6"]
                                  |  Statistics:Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
                                  |<-Map 11 [SIMPLE_EDGE]
@@ -584,7 +584,7 @@ Stage-0
                                              Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                                              Merge Join Operator [MERGEJOIN_54]
                                              |  condition map:[{"":"Inner Join 0 to 1"}]
-                                             |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                             |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                              |  outputColumnNames:["_col0","_col1","_col2","_col4"]
                                              |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                                              |<-Reducer 10 [SIMPLE_EDGE]
@@ -701,7 +701,7 @@ Stage-0
          File Output Operator [FS_44]
             compressed:false
             Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_43]
             |  outputColumnNames:["_col0","_col1","_col2"]
             |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
@@ -736,7 +736,7 @@ Stage-0
                                  Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                                  Merge Join Operator [MERGEJOIN_53]
                                  |  condition map:[{"":"Left Outer Join0 to 1"}]
-                                 |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                 |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                  |  outputColumnNames:["_col1","_col2","_col6"]
                                  |  Statistics:Num rows: 4 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                                  |<-Map 10 [SIMPLE_EDGE]
@@ -767,7 +767,7 @@ Stage-0
                                              Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                                              Merge Join Operator [MERGEJOIN_52]
                                              |  condition map:[{"":"Left Outer Join0 to 1"}]
-                                             |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                             |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                              |  outputColumnNames:["_col0","_col1","_col2","_col4"]
                                              |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                                              |<-Reducer 3 [SIMPLE_EDGE]
@@ -870,7 +870,7 @@ Stage-0
          File Output Operator [FS_36]
             compressed:false
             Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_34]
             |  aggregations:["count(VALUE._col0)"]
             |  keys:KEY._col0 (type: int), KEY._col1 (type: bigint)
@@ -896,7 +896,7 @@ Stage-0
                            Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                            Merge Join Operator [MERGEJOIN_41]
                            |  condition map:[{"":"Right Outer Join0 to 1"},{"":"Right Outer Join0 to 2"}]
-                           |  keys:{"2":"_col0 (type: string)","1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                           |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)","2":"_col0 (type: string)"}
                            |  outputColumnNames:["_col1","_col2","_col4","_col6"]
                            |  Statistics:Num rows: 4 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
                            |<-Map 8 [SIMPLE_EDGE]
@@ -1014,7 +1014,7 @@ Stage-0
          File Output Operator [FS_43]
             compressed:false
             Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_42]
             |  outputColumnNames:["_col0","_col1","_col2"]
             |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1049,7 +1049,7 @@ Stage-0
                                  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                                  Merge Join Operator [MERGEJOIN_48]
                                  |  condition map:[{"":"Outer Join 0 to 1"},{"":"Outer Join 0 to 2"}]
-                                 |  keys:{"2":"_col0 (type: string)","1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                 |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)","2":"_col0 (type: string)"}
                                  |  outputColumnNames:["_col1","_col2","_col4","_col6"]
                                  |  Statistics:Num rows: 4 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
                                  |<-Map 10 [SIMPLE_EDGE]
@@ -1177,7 +1177,7 @@ Stage-0
          File Output Operator [FS_41]
             compressed:false
             Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_39]
             |  aggregations:["count(VALUE._col0)"]
             |  keys:KEY._col0 (type: int), KEY._col1 (type: bigint)
@@ -1203,7 +1203,7 @@ Stage-0
                            Statistics:Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                            Merge Join Operator [MERGEJOIN_51]
                            |  condition map:[{"":"Inner Join 0 to 1"}]
-                           |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                           |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                            |  outputColumnNames:["_col1","_col2","_col6"]
                            |  Statistics:Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
                            |<-Map 8 [SIMPLE_EDGE]
@@ -1237,7 +1237,7 @@ Stage-0
                                        Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                                        Merge Join Operator [MERGEJOIN_50]
                                        |  condition map:[{"":"Inner Join 0 to 1"}]
-                                       |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                       |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                        |  outputColumnNames:["_col0","_col1","_col2","_col4"]
                                        |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                                        |<-Reducer 2 [SIMPLE_EDGE]
@@ -1327,7 +1327,7 @@ Stage-0
          File Output Operator [FS_8]
             compressed:false
             Statistics:Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_6]
                outputColumnNames:["_col0"]
                Statistics:Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1378,7 +1378,7 @@ Stage-0
          File Output Operator [FS_29]
             compressed:false
             Statistics:Num rows: 3 Data size: 261 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_28]
             |  outputColumnNames:["_col0"]
             |  Statistics:Num rows: 3 Data size: 261 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1494,7 +1494,7 @@ Stage-0
          File Output Operator [FS_34]
             compressed:false
             Statistics:Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_33]
             |  outputColumnNames:["_col0","_col1"]
             |  Statistics:Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1630,10 +1630,10 @@ Stage-0
          File Output Operator [FS_12]
             compressed:false
             Statistics:Num rows: 18 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Merge Join Operator [MERGEJOIN_17]
             |  condition map:[{"":"Inner Join 0 to 1"}]
-            |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+            |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
             |  outputColumnNames:["_col0"]
             |  Statistics:Num rows: 18 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 1 [SIMPLE_EDGE]
@@ -1684,13 +1684,13 @@ Stage-0
          File Output Operator [FS_8]
             compressed:false
             Statistics:Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_7]
                outputColumnNames:["_col0","_col1"]
                Statistics:Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                Merge Join Operator [MERGEJOIN_11]
                |  condition map:[{"":"Left Outer Join0 to 1"}]
-               |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+               |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                |  outputColumnNames:["_col1","_col3"]
                |  Statistics:Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 1 [SIMPLE_EDGE]
@@ -1737,13 +1737,13 @@ Stage-0
          File Output Operator [FS_8]
             compressed:false
             Statistics:Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_7]
                outputColumnNames:["_col0","_col1"]
                Statistics:Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                Merge Join Operator [MERGEJOIN_9]
                |  condition map:[{"":"Outer Join 0 to 1"}]
-               |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+               |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                |  outputColumnNames:["_col1","_col3"]
                |  Statistics:Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 1 [SIMPLE_EDGE]
@@ -1790,13 +1790,13 @@ Stage-0
          File Output Operator [FS_14]
             compressed:false
             Statistics:Num rows: 291 Data size: 29391 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_13]
                outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
                Statistics:Num rows: 291 Data size: 29391 Basic stats: COMPLETE Column stats: COMPLETE
                Merge Join Operator [MERGEJOIN_24]
                |  condition map:[{"":"Inner Join 0 to 1"},{"":"Inner Join 0 to 2"}]
-               |  keys:{"2":"_col0 (type: string)","1":"_col0 (type: string)","0":"_col0 (type: string)"}
+               |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)","2":"_col0 (type: string)"}
                |  outputColumnNames:["_col1","_col2","_col4","_col5","_col6"]
                |  Statistics:Num rows: 291 Data size: 29391 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 1 [SIMPLE_EDGE]
@@ -1865,13 +1865,13 @@ Stage-0
          File Output Operator [FS_14]
             compressed:false
             Statistics:Num rows: 291 Data size: 51798 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_13]
                outputColumnNames:["_col0","_col1","_col2","_col3"]
                Statistics:Num rows: 291 Data size: 51798 Basic stats: COMPLETE Column stats: COMPLETE
                Merge Join Operator [MERGEJOIN_24]
                |  condition map:[{"":"Inner Join 0 to 1"},{"":"Inner Join 0 to 2"}]
-               |  keys:{"2":"_col0 (type: string)","1":"_col0 (type: string)","0":"_col0 (type: string)"}
+               |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)","2":"_col0 (type: string)"}
                |  outputColumnNames:["_col0","_col1","_col3","_col4"]
                |  Statistics:Num rows: 291 Data size: 51798 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 1 [SIMPLE_EDGE]
@@ -1940,7 +1940,7 @@ Stage-0
          File Output Operator [FS_22]
             compressed:false
             Statistics:Num rows: 4 Data size: 404 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_19]
                outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
                Statistics:Num rows: 4 Data size: 404 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1949,7 +1949,7 @@ Stage-0
                   Statistics:Num rows: 4 Data size: 404 Basic stats: COMPLETE Column stats: COMPLETE
                   Merge Join Operator [MERGEJOIN_32]
                   |  condition map:[{"":"Inner Join 0 to 1"}]
-                  |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                  |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                   |  outputColumnNames:["_col1","_col2","_col3","_col4","_col6"]
                   |  Statistics:Num rows: 14 Data size: 1414 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Map 5 [SIMPLE_EDGE]
@@ -1980,7 +1980,7 @@ Stage-0
                            Statistics:Num rows: 4 Data size: 728 Basic stats: COMPLETE Column stats: COMPLETE
                            Merge Join Operator [MERGEJOIN_31]
                            |  condition map:[{"":"Outer Join 0 to 1"}]
-                           |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                           |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                            |  outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
                            |  Statistics:Num rows: 18 Data size: 3276 Basic stats: COMPLETE Column stats: COMPLETE
                            |<-Map 1 [SIMPLE_EDGE]
@@ -2033,7 +2033,7 @@ Stage-0
          File Output Operator [FS_16]
             compressed:false
             Statistics:Num rows: 12 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_13]
                outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
                Statistics:Num rows: 12 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2042,7 +2042,7 @@ Stage-0
                   Statistics:Num rows: 12 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
                   Merge Join Operator [MERGEJOIN_23]
                   |  condition map:[{"":"Right Outer Join0 to 1"},{"":"Right Outer Join0 to 2"}]
-                  |  keys:{"2":"_col0 (type: string)","1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                  |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)","2":"_col0 (type: string)"}
                   |  outputColumnNames:["_col1","_col2","_col3","_col4","_col6"]
                   |  Statistics:Num rows: 72 Data size: 7272 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Map 1 [SIMPLE_EDGE]
@@ -2109,7 +2109,7 @@ Stage-0
          File Output Operator [FS_10]
             compressed:false
             Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Limit [LIM_9]
                Number of rows:1
                Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2168,7 +2168,7 @@ Stage-0
          File Output Operator [FS_15]
             compressed:false
             Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Limit [LIM_14]
                Number of rows:1
                Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2246,7 +2246,7 @@ Stage-0
          File Output Operator [FS_13]
             compressed:false
             Statistics:Num rows: 5 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Limit [LIM_12]
                Number of rows:5
                Statistics:Num rows: 5 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2304,7 +2304,7 @@ Stage-0
          File Output Operator [FS_13]
             compressed:false
             Statistics:Num rows: 5 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Limit [LIM_12]
                Number of rows:5
                Statistics:Num rows: 5 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2372,7 +2372,7 @@ Stage-0
          File Output Operator [FS_48]
             compressed:false
             Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Limit [LIM_47]
                Number of rows:5
                Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2410,7 +2410,7 @@ Stage-0
                                     Statistics:Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                                     Merge Join Operator [MERGEJOIN_60]
                                     |  condition map:[{"":"Inner Join 0 to 1"}]
-                                    |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                    |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                     |  outputColumnNames:["_col1","_col2","_col6"]
                                     |  Statistics:Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
                                     |<-Map 11 [SIMPLE_EDGE]
@@ -2444,7 +2444,7 @@ Stage-0
                                                 Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                                                 Merge Join Operator [MERGEJOIN_59]
                                                 |  condition map:[{"":"Inner Join 0 to 1"}]
-                                                |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                                |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                                 |  outputColumnNames:["_col0","_col1","_col2","_col4"]
                                                 |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                                                 |<-Reducer 10 [SIMPLE_EDGE]
@@ -2567,13 +2567,13 @@ Stage-0
          File Output Operator [FS_13]
             compressed:false
             Statistics:Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_12]
                outputColumnNames:["_col0"]
                Statistics:Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                Merge Join Operator [MERGEJOIN_18]
                |  condition map:[{"":"Left Semi Join 0 to 1"}]
-               |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+               |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                |  outputColumnNames:["_col1"]
                |  Statistics:Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 1 [SIMPLE_EDGE]
@@ -2629,13 +2629,13 @@ Stage-0
          File Output Operator [FS_24]
             compressed:false
             Statistics:Num rows: 12 Data size: 1116 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_23]
                outputColumnNames:["_col0","_col1","_col2"]
                Statistics:Num rows: 12 Data size: 1116 Basic stats: COMPLETE Column stats: COMPLETE
                Merge Join Operator [MERGEJOIN_34]
                |  condition map:[{"":"Left Semi Join 0 to 1"},{"":"Left Semi Join 0 to 2"}]
-               |  keys:{"2":"_col0 (type: string)","1":"_col0 (type: string)","0":"_col0 (type: string)"}
+               |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)","2":"_col0 (type: string)"}
                |  outputColumnNames:["_col0","_col1","_col2"]
                |  Statistics:Num rows: 12 Data size: 1116 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 1 [SIMPLE_EDGE]
@@ -2716,7 +2716,7 @@ Stage-0
          File Output Operator [FS_53]
             compressed:false
             Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_52]
             |  outputColumnNames:["_col0","_col1","_col2"]
             |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2745,7 +2745,7 @@ Stage-0
                            Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                            Merge Join Operator [MERGEJOIN_66]
                            |  condition map:[{"":"Left Semi Join 0 to 1"},{"":"Left Semi Join 0 to 2"}]
-                           |  keys:{"2":"_col0 (type: string)","1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                           |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)","2":"_col0 (type: string)"}
                            |  outputColumnNames:["_col0","_col1"]
                            |  Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                            |<-Map 10 [SIMPLE_EDGE]
@@ -2931,7 +2931,7 @@ Stage-0
          File Output Operator [FS_16]
             compressed:false
             Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_14]
             |  aggregations:["count(VALUE._col0)"]
             |  outputColumnNames:["_col0"]
@@ -3001,7 +3001,7 @@ Stage-0
          File Output Operator [FS_16]
             compressed:false
             Statistics:Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_15]
                outputColumnNames:["_col0","_col1"]
                Statistics:Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3010,7 +3010,7 @@ Stage-0
                   Statistics:Num rows: 1 Data size: 269 Basic stats: COMPLETE Column stats: COMPLETE
                   Merge Join Operator [MERGEJOIN_20]
                   |  condition map:[{"":"Left Outer Join0 to 1"}]
-                  |  keys:{"1":"_col1 (type: string)","0":"_col1 (type: string)"}
+                  |  keys:{"0":"_col1 (type: string)","1":"_col1 (type: string)"}
                   |  outputColumnNames:["_col0","_col1","_col3"]
                   |  Statistics:Num rows: 193 Data size: 51917 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Map 1 [SIMPLE_EDGE]
@@ -3091,7 +3091,7 @@ Stage-0
          File Output Operator [FS_16]
             compressed:false
             Statistics:Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_15]
                outputColumnNames:["_col0","_col1"]
                Statistics:Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3100,7 +3100,7 @@ Stage-0
                   Statistics:Num rows: 1 Data size: 265 Basic stats: COMPLETE Column stats: COMPLETE
                   Merge Join Operator [MERGEJOIN_20]
                   |  condition map:[{"":"Left Outer Join0 to 1"}]
-                  |  keys:{"1":"_col0 (type: string), _col1 (type: string)","0":"_col1 (type: string), _col0 (type: string)"}
+                  |  keys:{"0":"_col1 (type: string), _col0 (type: string)","1":"_col0 (type: string), _col1 (type: string)"}
                   |  outputColumnNames:["_col0","_col1","_col3"]
                   |  Statistics:Num rows: 1 Data size: 265 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Map 4 [SIMPLE_EDGE]
@@ -3184,10 +3184,10 @@ Stage-0
          File Output Operator [FS_16]
             compressed:false
             Statistics:Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Merge Join Operator [MERGEJOIN_21]
             |  condition map:[{"":"Left Semi Join 0 to 1"}]
-            |  keys:{"1":"_col0 (type: string), _col1 (type: string)","0":"_col1 (type: string), _col0 (type: string)"}
+            |  keys:{"0":"_col1 (type: string), _col0 (type: string)","1":"_col0 (type: string), _col1 (type: string)"}
             |  outputColumnNames:["_col0","_col1"]
             |  Statistics:Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 1 [SIMPLE_EDGE]
@@ -3256,10 +3256,10 @@ Stage-0
          File Output Operator [FS_16]
             compressed:false
             Statistics:Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Merge Join Operator [MERGEJOIN_21]
             |  condition map:[{"":"Left Semi Join 0 to 1"}]
-            |  keys:{"1":"_col0 (type: string), _col1 (type: string)","0":"_col1 (type: string), _col0 (type: string)"}
+            |  keys:{"0":"_col1 (type: string), _col0 (type: string)","1":"_col0 (type: string), _col1 (type: string)"}
             |  outputColumnNames:["_col0","_col1"]
             |  Statistics:Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 1 [SIMPLE_EDGE]
@@ -3318,10 +3318,10 @@ Stage-0
          File Output Operator [FS_16]
             compressed:false
             Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Merge Join Operator [MERGEJOIN_21]
             |  condition map:[{"":"Left Semi Join 0 to 1"}]
-            |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+            |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
             |  outputColumnNames:["_col0","_col1"]
             |  Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 1 [SIMPLE_EDGE]
@@ -3385,13 +3385,13 @@ Stage-0
          File Output Operator [FS_28]
             compressed:false
             Statistics:Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_27]
                outputColumnNames:["_col0","_col1"]
                Statistics:Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                Merge Join Operator [MERGEJOIN_38]
                |  condition map:[{"":"Inner Join 0 to 1"}]
-               |  keys:{"1":"_col0 (type: int)","0":"_col1 (type: int)"}
+               |  keys:{"0":"_col1 (type: int)","1":"_col0 (type: int)"}
                |  outputColumnNames:["_col1","_col2"]
                |  Statistics:Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Reducer 2 [SIMPLE_EDGE]
@@ -3403,7 +3403,7 @@ Stage-0
                |     value expressions:_col2 (type: int)
                |     Merge Join Operator [MERGEJOIN_37]
                |     |  condition map:[{"":"Left Semi Join 0 to 1"}]
-               |     |  keys:{"1":"_col0 (type: int)","0":"_col0 (type: int)"}
+               |     |  keys:{"0":"_col0 (type: int)","1":"_col0 (type: int)"}
                |     |  outputColumnNames:["_col1","_col2"]
                |     |  Statistics:Num rows: 12 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
                |     |<-Map 1 [SIMPLE_EDGE]
@@ -3499,10 +3499,10 @@ Stage-0
          File Output Operator [FS_38]
             compressed:false
             Statistics:Num rows: 34 Data size: 6324 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Merge Join Operator [MERGEJOIN_51]
             |  condition map:[{"":"Left Semi Join 0 to 1"}]
-            |  keys:{"1":"_col0 (type: bigint)","0":"_col2 (type: bigint)"}
+            |  keys:{"0":"_col2 (type: bigint)","1":"_col0 (type: bigint)"}
             |  outputColumnNames:["_col0","_col1","_col2"]
             |  Statistics:Num rows: 34 Data size: 6324 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Reducer 3 [SIMPLE_EDGE]
@@ -3534,7 +3534,7 @@ Stage-0
             |                 Statistics:Num rows: 83 Data size: 15438 Basic stats: COMPLETE Column stats: COMPLETE
             |                 Merge Join Operator [MERGEJOIN_50]
             |                 |  condition map:[{"":"Left Semi Join 0 to 1"}]
-            |                 |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+            |                 |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
             |                 |  outputColumnNames:["_col0","_col1"]
             |                 |  Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
             |                 |<-Map 1 [SIMPLE_EDGE]
@@ -3645,10 +3645,10 @@ Stage-0
          File Output Operator [FS_22]
             compressed:false
             Statistics:Num rows: 3 Data size: 681 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Merge Join Operator [MERGEJOIN_27]
             |  condition map:[{"":"Left Semi Join 0 to 1"}]
-            |  keys:{"1":"_col0 (type: string)","0":"_col1 (type: string)"}
+            |  keys:{"0":"_col1 (type: string)","1":"_col0 (type: string)"}
             |  outputColumnNames:["_col0","_col1","_col2"]
             |  Statistics:Num rows: 3 Data size: 681 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Reducer 2 [SIMPLE_EDGE]
@@ -3701,7 +3701,7 @@ Stage-0
                            predicate:first_value_window_0 is not null (type: boolean)
                            Statistics:Num rows: 13 Data size: 6383 Basic stats: COMPLETE Column stats: COMPLETE
                            PTF Operator [PTF_11]
-                              Function definitions:[{"Input definition":{"type:":"WINDOWING"}},{"partition by:":"_col2","name:":"windowingtablefunction","order by:":"_col5"}]
+                              Function definitions:[{"Input definition":{"type:":"WINDOWING"}},{"name:":"windowingtablefunction","order by:":"_col5","partition by:":"_col2"}]
                               Statistics:Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE
                               Select Operator [SEL_10]
                               |  outputColumnNames:["_col1","_col2","_col5"]
@@ -3747,7 +3747,7 @@ Stage-0
          File Output Operator [FS_29]
             compressed:false
             Statistics:Num rows: 302 Data size: 53756 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_28]
             |  outputColumnNames:["_col0","_col1"]
             |  Statistics:Num rows: 302 Data size: 53756 Basic stats: COMPLETE Column stats: NONE
@@ -3765,7 +3765,7 @@ Stage-0
                         Statistics:Num rows: 302 Data size: 53756 Basic stats: COMPLETE Column stats: NONE
                         Merge Join Operator [MERGEJOIN_37]
                         |  condition map:[{"":"Left Outer Join0 to 1"}]
-                        |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                        |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                         |  outputColumnNames:["_col0","_col1","_col3"]
                         |  Statistics:Num rows: 605 Data size: 107690 Basic stats: COMPLETE Column stats: NONE
                         |<-Map 7 [SIMPLE_EDGE]
@@ -3868,7 +3868,7 @@ Stage-0
          File Output Operator [FS_27]
             compressed:false
             Statistics:Num rows: 15 Data size: 3507 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_26]
                outputColumnNames:["_col0","_col1","_col2"]
                Statistics:Num rows: 15 Data size: 3507 Basic stats: COMPLETE Column stats: NONE
@@ -3877,7 +3877,7 @@ Stage-0
                   Statistics:Num rows: 15 Data size: 3507 Basic stats: COMPLETE Column stats: NONE
                   Merge Join Operator [MERGEJOIN_35]
                   |  condition map:[{"":"Left Outer Join0 to 1"}]
-                  |  keys:{"1":"_col0 (type: string), _col1 (type: string)","0":"_col0 (type: string), _col1 (type: string)"}
+                  |  keys:{"0":"_col0 (type: string), _col1 (type: string)","1":"_col0 (type: string), _col1 (type: string)"}
                   |  outputColumnNames:["_col0","_col1","_col2","_col4"]
                   |  Statistics:Num rows: 30 Data size: 7014 Basic stats: COMPLETE Column stats: NONE
                   |<-Map 6 [SIMPLE_EDGE]
@@ -3982,7 +3982,7 @@ Stage-0
          File Output Operator [FS_39]
             compressed:false
             Statistics:Num rows: 1 Data size: 146 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_38]
             |  outputColumnNames:["_col0","_col1"]
             |  Statistics:Num rows: 1 Data size: 146 Basic stats: COMPLETE Column stats: NONE
@@ -4007,7 +4007,7 @@ Stage-0
                   |        Statistics:Num rows: 1 Data size: 133 Basic stats: COMPLETE Column stats: COMPLETE
                   |        Merge Join Operator [MERGEJOIN_48]
                   |        |  condition map:[{"":"Left Outer Join0 to 1"}]
-                  |        |  keys:{"1":"_col0 (type: double)","0":"UDFToDouble(_col1) (type: double)"}
+                  |        |  keys:{"0":"UDFToDouble(_col1) (type: double)","1":"_col0 (type: double)"}
                   |        |  outputColumnNames:["_col0","_col1","_col2"]
                   |        |  Statistics:Num rows: 1 Data size: 133 Basic stats: COMPLETE Column stats: COMPLETE
                   |        |<-Map 1 [SIMPLE_EDGE]
@@ -4131,7 +4131,7 @@ Stage-0
          File Output Operator [FS_44]
             compressed:false
             Statistics:Num rows: 2 Data size: 256 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_43]
             |  outputColumnNames:["_col0","_col1"]
             |  Statistics:Num rows: 2 Data size: 256 Basic stats: COMPLETE Column stats: NONE
@@ -4149,7 +4149,7 @@ Stage-0
                         Statistics:Num rows: 2 Data size: 256 Basic stats: COMPLETE Column stats: NONE
                         Merge Join Operator [MERGEJOIN_53]
                         |  condition map:[{"":"Left Outer Join0 to 1"}]
-                        |  keys:{"1":"_col0 (type: string), _col1 (type: double)","0":"_col0 (type: string), _col1 (type: double)"}
+                        |  keys:{"0":"_col0 (type: string), _col1 (type: double)","1":"_col0 (type: string), _col1 (type: double)"}
                         |  outputColumnNames:["_col0","_col1","_col3"]
                         |  Statistics:Num rows: 5 Data size: 641 Basic stats: COMPLETE Column stats: NONE
                         |<-Reducer 10 [SIMPLE_EDGE]
@@ -4294,12 +4294,12 @@ Stage-0
          File Output Operator [FS_6]
             compressed:false
             Statistics:Num rows: 20 Data size: 1040 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_4]
                outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
                Statistics:Num rows: 20 Data size: 1040 Basic stats: COMPLETE Column stats: COMPLETE
                PTF Operator [PTF_3]
-                  Function definitions:[{"Input definition":{"type:":"WINDOWING"}},{"partition by:":"0","name:":"windowingtablefunction","order by:":"0"}]
+                  Function definitions:[{"Input definition":{"type:":"WINDOWING"}},{"name:":"windowingtablefunction","order by:":"0","partition by:":"0"}]
                   Statistics:Num rows: 20 Data size: 9184 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator [SEL_2]
                   |  outputColumnNames:["_col2","_col3"]
@@ -4332,12 +4332,12 @@ Stage-0
          File Output Operator [FS_6]
             compressed:false
             Statistics:Num rows: 20 Data size: 1040 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_4]
                outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
                Statistics:Num rows: 20 Data size: 1040 Basic stats: COMPLETE Column stats: COMPLETE
                PTF Operator [PTF_3]
-                  Function definitions:[{"Input definition":{"type:":"WINDOWING"}},{"partition by:":"0","name:":"windowingtablefunction","order by:":"0"}]
+                  Function definitions:[{"Input definition":{"type:":"WINDOWING"}},{"name:":"windowingtablefunction","order by:":"0","partition by:":"0"}]
                   Statistics:Num rows: 20 Data size: 9184 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator [SEL_2]
                   |  outputColumnNames:["_col2","_col3"]
@@ -4370,12 +4370,12 @@ Stage-0
          File Output Operator [FS_6]
             compressed:false
             Statistics:Num rows: 20 Data size: 1280 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_4]
                outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10"]
                Statistics:Num rows: 20 Data size: 1280 Basic stats: COMPLETE Column stats: COMPLETE
                PTF Operator [PTF_3]
-                  Function definitions:[{"Input definition":{"type:":"WINDOWING"}},{"partition by:":"_col0","name:":"windowingtablefunction","order by:":"_col1"}]
+                  Function definitions:[{"Input definition":{"type:":"WINDOWING"}},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col0"}]
                   Statistics:Num rows: 20 Data size: 12244 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator [SEL_2]
                   |  outputColumnNames:["_col0","_col1","_col2","_col3"]
@@ -4408,12 +4408,12 @@ Stage-0
          File Output Operator [FS_6]
             compressed:false
             Statistics:Num rows: 25 Data size: 4475 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_4]
                outputColumnNames:["_col0","_col1","_col2"]
                Statistics:Num rows: 25 Data size: 4475 Basic stats: COMPLETE Column stats: COMPLETE
                PTF Operator [PTF_3]
-                  Function definitions:[{"Input definition":{"type:":"WINDOWING"}},{"partition by:":"_col0","name:":"windowingtablefunction","order by:":"_col1"}]
+                  Function definitions:[{"Input definition":{"type:":"WINDOWING"}},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col0"}]
                   Statistics:Num rows: 25 Data size: 11075 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator [SEL_2]
                   |  outputColumnNames:["_col0","_col1"]
@@ -4455,7 +4455,7 @@ Stage-0
          File Output Operator [FS_20]
             compressed:false
             Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_18]
             |  aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
             |  outputColumnNames:["_col0","_col1"]
@@ -4494,7 +4494,7 @@ Stage-0
                                     Statistics:Num rows: 60 Data size: 5160 Basic stats: COMPLETE Column stats: COMPLETE
                                     Merge Join Operator [MERGEJOIN_25]
                                     |  condition map:[{"":"Inner Join 0 to 1"}]
-                                    |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                    |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                     |  outputColumnNames:["_col1"]
                                     |  Statistics:Num rows: 60 Data size: 5160 Basic stats: COMPLETE Column stats: COMPLETE
                                     |<-Map 1 [SIMPLE_EDGE]
@@ -4555,7 +4555,7 @@ Stage-0
          File Output Operator [FS_20]
             compressed:false
             Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_18]
             |  aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
             |  outputColumnNames:["_col0","_col1"]
@@ -4594,7 +4594,7 @@ Stage-0
                                     Statistics:Num rows: 60 Data size: 5160 Basic stats: COMPLETE Column stats: COMPLETE
                                     Merge Join Operator [MERGEJOIN_25]
                                     |  condition map:[{"":"Inner Join 0 to 1"}]
-                                    |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                    |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                     |  outputColumnNames:["_col1"]
                                     |  Statistics:Num rows: 60 Data size: 5160 Basic stats: COMPLETE Column stats: COMPLETE
                                     |<-Map 1 [SIMPLE_EDGE]
@@ -4655,7 +4655,7 @@ Stage-0
          File Output Operator [FS_20]
             compressed:false
             Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_18]
             |  aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
             |  outputColumnNames:["_col0","_col1"]
@@ -4749,7 +4749,7 @@ Stage-0
          File Output Operator [FS_22]
             compressed:false
             Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_20]
             |  aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
             |  outputColumnNames:["_col0","_col1"]
@@ -4785,7 +4785,7 @@ Stage-0
                                  Statistics:Num rows: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE
                                  Merge Join Operator [MERGEJOIN_27]
                                  |  condition map:[{"":"Left Semi Join 0 to 1"}]
-                                 |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                 |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                  |  outputColumnNames:["_col0"]
                                  |  Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                                  |<-Map 1 [SIMPLE_EDGE]
@@ -4867,7 +4867,7 @@ Stage-0
          File Output Operator [FS_7]
             compressed:false
             Statistics:Num rows: 2 Data size: 39 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_5]
             |  aggregations:["count(DISTINCT KEY._col1:0._col0)","count(DISTINCT KEY._col1:1._col0)","sum(VALUE._col2)"]
             |  keys:KEY._col0 (type: int)
@@ -4909,7 +4909,7 @@ Stage-0
          File Output Operator [FS_6]
             compressed:false
             Statistics:Num rows: 2 Data size: 39 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_4]
             |  aggregations:["count(DISTINCT KEY._col1:0._col0)","count(DISTINCT KEY._col1:1._col0)","sum(VALUE._col0)"]
             |  keys:KEY._col0 (type: int)
@@ -5019,7 +5019,7 @@ Stage-0
          File Output Operator [FS_6]
             compressed:true
             Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_4]
             |  aggregations:["count(1)"]
             |  outputColumnNames:["_col0"]
@@ -5051,7 +5051,7 @@ Stage-0
          File Output Operator [FS_6]
             compressed:true
             Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_4]
             |  aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
             |  outputColumnNames:["_col0","_col1"]
@@ -5110,7 +5110,7 @@ Stage-0
          File Output Operator [FS_6]
             compressed:true
             Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_4]
             |  aggregations:["count(1)"]
             |  outputColumnNames:["_col0"]
@@ -5142,7 +5142,7 @@ Stage-0
          File Output Operator [FS_6]
             compressed:true
             Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_4]
             |  aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
             |  outputColumnNames:["_col0","_col1"]
@@ -5192,7 +5192,7 @@ Stage-0
          File Output Operator [FS_9]
             compressed:true
             Statistics:Num rows: 125000 Data size: 10875000 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Merge Join Operator [MERGEJOIN_11]
             |  condition map:[{"":"Inner Join 0 to 1"}]
             |  keys:{}
@@ -5263,7 +5263,7 @@ Stage-3
                      File Output Operator [FS_8]
                         compressed:true
                         Statistics:Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
-                        table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","name:":"default.nzhang_CTAS1","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+                        table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","name:":"default.nzhang_CTAS1"}
                         Limit [LIM_7]
                            Number of rows:10
                            Statistics:Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
@@ -5331,7 +5331,7 @@ Stage-3
                      File Output Operator [FS_8]
                         compressed:true
                         Statistics:Num rows: 10 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE
-                        table:{"serde:":"org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe","name:":"default.nzhang_ctas3","input format:":"org.apache.hadoop.hive.ql.io.RCFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.RCFileOutputFormat"}
+                        table:{"input format:":"org.apache.hadoop.hive.ql.io.RCFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.RCFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe","name:":"default.nzhang_ctas3"}
                         Limit [LIM_7]
                            Number of rows:10
                            Statistics:Num rows: 10 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE
@@ -5434,7 +5434,7 @@ Stage-0
          File Output Operator [FS_12]
             compressed:true
             Statistics:Num rows: 13778 Data size: 4904968 Basic stats: COMPLETE Column stats: COMPLETE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_11]
             |  outputColumnNames:["_col0","_col1","_col2","_col3"]
             |  Statistics:Num rows: 13778 Data size: 4904968 Basic stats: COMPLETE Column stats: COMPLETE
@@ -5510,13 +5510,13 @@ Stage-0
          File Output Operator [FS_6]
             compressed:true
             Statistics:Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_5]
                outputColumnNames:["_col0","_col1","_col2","_col3"]
                Statistics:Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE
                Merge Join Operator [MERGEJOIN_7]
                |  condition map:[{"":"Inner Join 0 to 1"}]
-               |  keys:{"1":"value (type: int)","0":"key (type: int)"}
+               |  keys:{"0":"key (type: int)","1":"value (type: int)"}
                |  outputColumnNames:["_col0","_col1","_col5","_col6"]
                |  Statistics:Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE
                |<-Map 1 [SIMPLE_EDGE]
@@ -5557,13 +5557,13 @@ Stage-0
          File Output Operator [FS_11]
             compressed:true
             Statistics:Num rows: 4 Data size: 37 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_10]
                outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
                Statistics:Num rows: 4 Data size: 37 Basic stats: COMPLETE Column stats: NONE
                Merge Join Operator [MERGEJOIN_21]
                |  condition map:[{"":"Inner Join 0 to 1"},{"":"Inner Join 0 to 2"}]
-               |  keys:{"2":"key (type: int)","1":"value (type: int)","0":"key (type: int)"}
+               |  keys:{"0":"key (type: int)","1":"value (type: int)","2":"key (type: int)"}
                |  outputColumnNames:["_col0","_col1","_col5","_col6","_col10","_col11"]
                |  Statistics:Num rows: 4 Data size: 37 Basic stats: COMPLETE Column stats: NONE
                |<-Map 1 [SIMPLE_EDGE]
@@ -5623,13 +5623,13 @@ Stage-0
          File Output Operator [FS_8]
             compressed:true
             Statistics:Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_7]
                outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
                Statistics:Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE
                Merge Join Operator [MERGEJOIN_9]
                |  condition map:[{"":"Inner Join 0 to 1"},{"":"Inner Join 0 to 2"}]
-               |  keys:{"2":"key (type: int)","1":"value (type: int)","0":"key (type: int)"}
+               |  keys:{"0":"key (type: int)","1":"value (type: int)","2":"key (type: int)"}
                |  outputColumnNames:["_col0","_col1","_col5","_col6","_col10","_col11"]
                |  Statistics:Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE
                |<-Map 1 [SIMPLE_EDGE]
@@ -5680,13 +5680,13 @@ Stage-0
          File Output Operator [FS_11]
             compressed:true
             Statistics:Num rows: 4 Data size: 37 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_10]
                outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
                Statistics:Num rows: 4 Data size: 37 Basic stats: COMPLETE Column stats: NONE
                Merge Join Operator [MERGEJOIN_15]
                |  condition map:[{"":"Inner Join 0 to 1"},{"":"Inner Join 0 to 2"}]
-               |  keys:{"2":"key (type: int), value (type: int)","1":"value (type: int), key (type: int)","0":"key (type: int), value (type: int)"}
+               |  keys:{"0":"key (type: int), value (type: int)","1":"value (type: int), key (type: int)","2":"key (type: int), value (type: int)"}
                |  outputColumnNames:["_col0","_col1","_col5","_col6","_col10","_col11"]
                |  Statistics:Num rows: 4 Data size: 37 Basic stats: COMPLETE Column stats: NONE
                |<-Map 1 [SIMPLE_EDGE]
@@ -5743,13 +5743,13 @@ Stage-0
          File Output Operator [FS_8]
             compressed:true
             Statistics:Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_7]
                outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
                Statistics:Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE
                Merge Join Operator [MERGEJOIN_9]
                |  condition map:[{"":"Inner Join 0 to 1"},{"":"Inner Join 0 to 2"}]
-               |  keys:{"2":"key (type: int), value (type: int)","1":"value (type: int), key (type: int)","0":"key (type: int), value (type: int)"}
+               |  keys:{"0":"key (type: int), value (type: int)","1":"value (type: int), key (type: int)","2":"key (type: int), value (type: int)"}
                |  outputColumnNames:["_col0","_col1","_col5","_col6","_col10","_col11"]
                |  Statistics:Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE
                |<-Map 1 [SIMPLE_EDGE]
@@ -5797,13 +5797,13 @@ Stage-0
          File Output Operator [FS_6]
             compressed:true
             Statistics:Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_5]
                outputColumnNames:["_col0","_col1","_col2","_col3"]
                Statistics:Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE
                Merge Join Operator [MERGEJOIN_7]
                |  condition map:[{"":"Left Outer Join0 to 1"}]
-               |  keys:{"1":"value (type: int)","0":"key (type: int)"}
+               |  keys:{"0":"key (type: int)","1":"value (type: int)"}
                |  outputColumnNames:[

<TRUNCATED>

[48/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/correlationoptimizer6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/correlationoptimizer6.q.out b/ql/src/test/results/clientpositive/correlationoptimizer6.q.out
index 6b003d5..2d72250 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer6.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer6.q.out
@@ -2650,22 +2650,18 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -2706,22 +2702,18 @@ STAGE PLANS:
             1 _col0 (type: string)
           outputColumnNames: _col0, _col1, _col3
           Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: bigint), _col3 (type: bigint)
+          Group By Operator
+            aggregations: sum(_col1), sum(_col3)
+            keys: _col0 (type: string)
+            mode: hash
             outputColumnNames: _col0, _col1, _col2
             Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-            Group By Operator
-              aggregations: sum(_col1), sum(_col2)
-              keys: _col0 (type: string)
-              mode: hash
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
-                table:
-                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-3
     Map Reduce
@@ -2803,22 +2795,18 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -2930,44 +2918,36 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
           TableScan
             alias: y
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
           TableScan
             alias: x
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -3003,39 +2983,35 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3
                 Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: string), _col1 (type: bigint), _col3 (type: bigint)
-                  outputColumnNames: _col0, _col1, _col2
+                Mux Operator
                   Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                  Mux Operator
-                    Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                    Group By Operator
-                      aggregations: sum(_col1), sum(_col2)
-                      keys: _col0 (type: string)
-                      mode: complete
-                      outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                      Mux Operator
-                        Statistics: Num rows: 514 Data size: 5411 Basic stats: COMPLETE Column stats: NONE
-                        Join Operator
-                          condition map:
-                               Inner Join 0 to 1
-                          keys:
-                            0 _col0 (type: string)
-                            1 _col0 (type: string)
-                          outputColumnNames: _col0, _col1, _col2, _col4
+                  Group By Operator
+                    aggregations: sum(_col1), sum(_col3)
+                    keys: _col0 (type: string)
+                    mode: complete
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Mux Operator
+                      Statistics: Num rows: 514 Data size: 5411 Basic stats: COMPLETE Column stats: NONE
+                      Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0, _col1, _col2, _col4
+                        Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+                        Select Operator
+                          expressions: _col0 (type: string), _col1 (type: bigint), _col2 (type: bigint), _col0 (type: string), _col4 (type: string)
+                          outputColumnNames: _col0, _col1, _col2, _col3, _col4
                           Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                          Select Operator
-                            expressions: _col0 (type: string), _col1 (type: bigint), _col2 (type: bigint), _col0 (type: string), _col4 (type: string)
-                            outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                          File Output Operator
+                            compressed: false
                             Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                            File Output Operator
-                              compressed: false
-                              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                              table:
-                                  input format: org.apache.hadoop.mapred.TextInputFormat
-                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                            table:
+                                input format: org.apache.hadoop.mapred.TextInputFormat
+                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           Group By Operator
             aggregations: count(VALUE._col0)
             keys: KEY._col0 (type: string)
@@ -3052,39 +3028,35 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3
                 Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: string), _col1 (type: bigint), _col3 (type: bigint)
-                  outputColumnNames: _col0, _col1, _col2
+                Mux Operator
                   Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                  Mux Operator
-                    Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                    Group By Operator
-                      aggregations: sum(_col1), sum(_col2)
-                      keys: _col0 (type: string)
-                      mode: complete
-                      outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                      Mux Operator
-                        Statistics: Num rows: 514 Data size: 5411 Basic stats: COMPLETE Column stats: NONE
-                        Join Operator
-                          condition map:
-                               Inner Join 0 to 1
-                          keys:
-                            0 _col0 (type: string)
-                            1 _col0 (type: string)
-                          outputColumnNames: _col0, _col1, _col2, _col4
+                  Group By Operator
+                    aggregations: sum(_col1), sum(_col3)
+                    keys: _col0 (type: string)
+                    mode: complete
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Mux Operator
+                      Statistics: Num rows: 514 Data size: 5411 Basic stats: COMPLETE Column stats: NONE
+                      Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0, _col1, _col2, _col4
+                        Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+                        Select Operator
+                          expressions: _col0 (type: string), _col1 (type: bigint), _col2 (type: bigint), _col0 (type: string), _col4 (type: string)
+                          outputColumnNames: _col0, _col1, _col2, _col3, _col4
                           Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                          Select Operator
-                            expressions: _col0 (type: string), _col1 (type: bigint), _col2 (type: bigint), _col0 (type: string), _col4 (type: string)
-                            outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                          File Output Operator
+                            compressed: false
                             Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                            File Output Operator
-                              compressed: false
-                              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                              table:
-                                  input format: org.apache.hadoop.mapred.TextInputFormat
-                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                            table:
+                                input format: org.apache.hadoop.mapred.TextInputFormat
+                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           Mux Operator
             Statistics: Num rows: 514 Data size: 5411 Basic stats: COMPLETE Column stats: NONE
             Join Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/count.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/count.q.out b/ql/src/test/results/clientpositive/count.q.out
index e8d0cb3..c744b4f 100644
--- a/ql/src/test/results/clientpositive/count.q.out
+++ b/ql/src/test/results/clientpositive/count.q.out
@@ -48,11 +48,11 @@ STAGE PLANS:
             Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: int), b (type: int), c (type: int), d (type: int)
-              outputColumnNames: _col0, _col1, _col2, _col3
+              outputColumnNames: a, b, c, d
               Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1), count(DISTINCT _col2), sum(_col3)
-                keys: _col0 (type: int), _col1 (type: int), _col2 (type: int)
+                aggregations: count(DISTINCT b), count(DISTINCT c), sum(d)
+                keys: a (type: int), b (type: int), c (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                 Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
@@ -171,14 +171,14 @@ STAGE PLANS:
             Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: int), b (type: int), c (type: int), d (type: int)
-              outputColumnNames: _col0, _col1, _col2, _col3
+              outputColumnNames: a, b, c, d
               Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int)
+                key expressions: a (type: int), b (type: int), c (type: int)
                 sort order: +++
-                Map-reduce partition columns: _col0 (type: int)
+                Map-reduce partition columns: a (type: int)
                 Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
-                value expressions: _col3 (type: int)
+                value expressions: d (type: int)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(DISTINCT KEY._col1:0._col0), count(DISTINCT KEY._col1:1._col0), sum(VALUE._col0)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/ctas_colname.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ctas_colname.q.out b/ql/src/test/results/clientpositive/ctas_colname.q.out
index 232d505..2a70faf 100644
--- a/ql/src/test/results/clientpositive/ctas_colname.q.out
+++ b/ql/src/test/results/clientpositive/ctas_colname.q.out
@@ -692,11 +692,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: count(value)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1125,22 +1125,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (UDFToDouble(key) < 9.0) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
+              Group By Operator
+                aggregations: count(value)
+                keys: key (type: string), value (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count(_col1)
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col2 (type: bigint)
+                  value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -1256,22 +1252,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (UDFToDouble(key) < 9.0) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                aggregations: max(value)
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: max(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: string)
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Group By Operator
           aggregations: max(VALUE._col0)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/database.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/database.q.out b/ql/src/test/results/clientpositive/database.q.out
index 19b0ea2..02eb3d3 100644
--- a/ql/src/test/results/clientpositive/database.q.out
+++ b/ql/src/test/results/clientpositive/database.q.out
@@ -455,7 +455,7 @@ POSTHOOK: query: INSERT OVERWRITE TABLE temp_tbl2 SELECT COUNT(*) FROM temp_tbl
 POSTHOOK: type: QUERY
 POSTHOOK: Input: to_drop_db2@temp_tbl
 POSTHOOK: Output: to_drop_db2@temp_tbl2
-POSTHOOK: Lineage: temp_tbl2.c EXPRESSION []
+POSTHOOK: Lineage: temp_tbl2.c EXPRESSION [(temp_tbl)temp_tbl.null, ]
 PREHOOK: query: USE default
 PREHOOK: type: SWITCHDATABASE
 PREHOOK: Input: database:default

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/decimal_precision.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/decimal_precision.q.out b/ql/src/test/results/clientpositive/decimal_precision.q.out
index 8d67e2a..69a6045 100644
--- a/ql/src/test/results/clientpositive/decimal_precision.q.out
+++ b/ql/src/test/results/clientpositive/decimal_precision.q.out
@@ -539,10 +539,10 @@ STAGE PLANS:
             Statistics: Num rows: 23 Data size: 2661 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: dec (type: decimal(20,10))
-              outputColumnNames: _col0
+              outputColumnNames: dec
               Statistics: Num rows: 23 Data size: 2661 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: avg(_col0), sum(_col0)
+                aggregations: avg(dec), sum(dec)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/decimal_udf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/decimal_udf.q.out b/ql/src/test/results/clientpositive/decimal_udf.q.out
index 338b754..ce1fe3f 100644
--- a/ql/src/test/results/clientpositive/decimal_udf.q.out
+++ b/ql/src/test/results/clientpositive/decimal_udf.q.out
@@ -1291,11 +1291,11 @@ STAGE PLANS:
             Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: value (type: int), key (type: decimal(20,10))
-              outputColumnNames: _col0, _col1
+              outputColumnNames: value, key
               Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col1), count(_col1), avg(_col1)
-                keys: _col0 (type: int)
+                aggregations: sum(key), count(key), avg(key)
+                keys: value (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
@@ -1878,11 +1878,11 @@ STAGE PLANS:
             Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: value (type: int), key (type: decimal(20,10))
-              outputColumnNames: _col0, _col1
+              outputColumnNames: value, key
               Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: stddev(_col1), variance(_col1)
-                keys: _col0 (type: int)
+                aggregations: stddev(key), variance(key)
+                keys: value (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
@@ -1957,11 +1957,11 @@ STAGE PLANS:
             Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: value (type: int), key (type: decimal(20,10))
-              outputColumnNames: _col0, _col1
+              outputColumnNames: value, key
               Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: stddev_samp(_col1), var_samp(_col1)
-                keys: _col0 (type: int)
+                aggregations: stddev_samp(key), var_samp(key)
+                keys: value (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
@@ -2095,10 +2095,10 @@ STAGE PLANS:
             Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: decimal(20,10))
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: min(_col0)
+                aggregations: min(key)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
@@ -2154,10 +2154,10 @@ STAGE PLANS:
             Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: decimal(20,10))
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: max(_col0)
+                aggregations: max(key)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
@@ -2213,10 +2213,10 @@ STAGE PLANS:
             Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: decimal(20,10))
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 3 Data size: 359 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/distinct_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/distinct_stats.q.out b/ql/src/test/results/clientpositive/distinct_stats.q.out
index 36049be..164a0a8 100644
--- a/ql/src/test/results/clientpositive/distinct_stats.q.out
+++ b/ql/src/test/results/clientpositive/distinct_stats.q.out
@@ -43,11 +43,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: string), b (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: a, b
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT b)
+                keys: a (type: string), b (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -100,10 +100,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: b (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: b
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string)
+                keys: b (type: string)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -151,11 +151,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: a
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: a (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
index 216a79c..2b5122f 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
@@ -1518,35 +1518,35 @@ STAGE PLANS:
             Filter Operator
               predicate: (t is null or (t = 27)) (type: boolean)
               Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
+              Group By Operator
+                keys: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float)
+                mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                Reduce Output Operator
+                  key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
+                  sort order: +++++
+                  Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
                   Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                    sort order: +++++
-                    Map-reduce partition columns: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                    Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
-          keys: KEY._col0 (type: smallint), KEY._col1 (type: int), KEY._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint)
+          keys: KEY._col0 (type: tinyint), KEY._col1 (type: smallint), KEY._col2 (type: int), KEY._col3 (type: bigint), KEY._col4 (type: float)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
+          Select Operator
+            expressions: _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col0 (type: tinyint)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4
             Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-            table:
-                input format: org.apache.hadoop.mapred.TextInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                name: default.over1k_part2_orc
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.over1k_part2_orc
 
   Stage: Stage-0
     Move Operator
@@ -1572,9 +1572,8 @@ explain insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
-  Stage-3 depends on stages: Stage-0
+  Stage-0 depends on stages: Stage-1
+  Stage-2 depends on stages: Stage-0
 
 STAGE PLANS:
   Stage: Stage-1
@@ -1586,57 +1585,35 @@ STAGE PLANS:
             Filter Operator
               predicate: (t is null or (t = 27)) (type: boolean)
               Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
+              Group By Operator
+                keys: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float)
+                mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                Reduce Output Operator
+                  key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
+                  sort order: +++++
+                  Map-reduce partition columns: _col0 (type: tinyint)
                   Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                    sort order: +++++
-                    Map-reduce partition columns: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                    Statistics: Num rows: 1048 Data size: 310873 Basic stats: COMPLETE Column stats: NONE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
-          keys: KEY._col0 (type: smallint), KEY._col1 (type: int), KEY._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint)
+          keys: KEY._col0 (type: tinyint), KEY._col1 (type: smallint), KEY._col2 (type: int), KEY._col3 (type: bigint), KEY._col4 (type: float)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            table:
-                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col4 (type: tinyint)
-              sort order: +
-              Map-reduce partition columns: _col4 (type: tinyint)
-              Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-              value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-      Reduce Operator Tree:
-        Select Operator
-          expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: tinyint)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
+          Select Operator
+            expressions: _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col0 (type: tinyint)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4
             Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
-            table:
-                input format: org.apache.hadoop.mapred.TextInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                name: default.over1k_part2_orc
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.over1k_part2_orc
 
   Stage: Stage-0
     Move Operator
@@ -1651,7 +1628,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: default.over1k_part2_orc
 
-  Stage: Stage-3
+  Stage: Stage-2
     Stats-Aggr Operator
 
 PREHOOK: query: insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
index 41049bd..4caa587 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
@@ -1420,34 +1420,34 @@ STAGE PLANS:
             Filter Operator
               predicate: (t is null or (t = 27)) (type: boolean)
               Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
+              Group By Operator
+                keys: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float)
+                mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                Reduce Output Operator
+                  key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
+                  sort order: +++++
+                  Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
                   Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                    sort order: +++++
-                    Map-reduce partition columns: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                    Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
-          keys: KEY._col0 (type: smallint), KEY._col1 (type: int), KEY._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint)
+          keys: KEY._col0 (type: tinyint), KEY._col1 (type: smallint), KEY._col2 (type: int), KEY._col3 (type: bigint), KEY._col4 (type: float)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
+          Select Operator
+            expressions: _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col0 (type: tinyint)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4
             Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-            table:
-                input format: org.apache.hadoop.mapred.TextInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                name: default.over1k_part2
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.over1k_part2
 
   Stage: Stage-0
     Move Operator
@@ -1473,9 +1473,8 @@ explain insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
-  Stage-3 depends on stages: Stage-0
+  Stage-0 depends on stages: Stage-1
+  Stage-2 depends on stages: Stage-0
 
 STAGE PLANS:
   Stage: Stage-1
@@ -1487,56 +1486,34 @@ STAGE PLANS:
             Filter Operator
               predicate: (t is null or (t = 27)) (type: boolean)
               Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
+              Group By Operator
+                keys: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float)
+                mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                Reduce Output Operator
+                  key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float)
+                  sort order: +++++
+                  Map-reduce partition columns: _col0 (type: tinyint)
                   Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                    sort order: +++++
-                    Map-reduce partition columns: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-                    Statistics: Num rows: 4442 Data size: 106611 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
-          keys: KEY._col0 (type: smallint), KEY._col1 (type: int), KEY._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint)
+          keys: KEY._col0 (type: tinyint), KEY._col1 (type: smallint), KEY._col2 (type: int), KEY._col3 (type: bigint), KEY._col4 (type: float)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            table:
-                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col4 (type: tinyint)
-              sort order: +
-              Map-reduce partition columns: _col4 (type: tinyint)
-              Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-              value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
-      Reduce Operator Tree:
-        Select Operator
-          expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: tinyint)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
+          Select Operator
+            expressions: _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col0 (type: tinyint)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4
             Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
-            table:
-                input format: org.apache.hadoop.mapred.TextInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                name: default.over1k_part2
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 2221 Data size: 53305 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.over1k_part2
 
   Stage: Stage-0
     Move Operator
@@ -1551,7 +1528,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: default.over1k_part2
 
-  Stage: Stage-3
+  Stage: Stage-2
     Stats-Aggr Operator
 
 PREHOOK: query: insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_encrypted_tbl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_encrypted_tbl.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_encrypted_tbl.q.out
index 5d4d2af..c57a5ad 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_encrypted_tbl.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_encrypted_tbl.q.out
@@ -29,11 +29,11 @@ POSTHOOK: Output: default@encrypted_table
 PREHOOK: query: SELECT count(*) FROM encrypted_table
 PREHOOK: type: QUERY
 PREHOOK: Input: default@encrypted_table
-#### A masked pattern was here ####
+#### A PARTIAL masked pattern was here #### data/warehouse/default/encrypted_table/.hive-staging
 POSTHOOK: query: SELECT count(*) FROM encrypted_table
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@encrypted_table
-#### A masked pattern was here ####
+#### A PARTIAL masked pattern was here #### data/warehouse/default/encrypted_table/.hive-staging
 500
 PREHOOK: query: drop table encrypted_table PURGE
 PREHOOK: type: DROPTABLE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/explain_logical.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_logical.q.out b/ql/src/test/results/clientpositive/explain_logical.q.out
index 545034a..8fa0a4c 100644
--- a/ql/src/test/results/clientpositive/explain_logical.q.out
+++ b/ql/src/test/results/clientpositive/explain_logical.q.out
@@ -804,20 +804,20 @@ $hdt$_0:src
   TableScan (TS_0)
     alias: src
     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-    Filter Operator (FIL_18)
+    Filter Operator (FIL_17)
       predicate: key is not null (type: boolean)
       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
       Select Operator (SEL_1)
         expressions: key (type: string), value (type: string)
         outputColumnNames: _col0, _col1
         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-        Reduce Output Operator (RS_10)
+        Reduce Output Operator (RS_9)
           key expressions: _col0 (type: string)
           sort order: +
           Map-reduce partition columns: _col0 (type: string)
           Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
           value expressions: _col1 (type: string)
-          Join Operator (JOIN_13)
+          Join Operator (JOIN_12)
             condition map:
                  Inner Join 0 to 1
             keys:
@@ -825,67 +825,63 @@ $hdt$_0:src
               1 _col0 (type: string)
             outputColumnNames: _col0, _col1, _col3
             Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-            Select Operator (SEL_14)
+            Select Operator (SEL_13)
               expressions: _col0 (type: string), _col3 (type: bigint), _col1 (type: string)
               outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator (RS_15)
+              Reduce Output Operator (RS_14)
                 key expressions: _col0 (type: string)
                 sort order: +
                 Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
                 value expressions: _col1 (type: bigint), _col2 (type: string)
-                Select Operator (SEL_16)
+                Select Operator (SEL_15)
                   expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint), VALUE._col1 (type: string)
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator (FS_17)
+                  File Output Operator (FS_16)
                     compressed: false
                     Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-$hdt$_1:$hdt$_1:src 
+$hdt$_1:src 
   TableScan (TS_2)
     alias: src
     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-    Filter Operator (FIL_19)
+    Filter Operator (FIL_18)
       predicate: key is not null (type: boolean)
       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-      Select Operator (SEL_3)
-        expressions: key (type: string), value (type: string)
+      Group By Operator (GBY_4)
+        aggregations: count(value)
+        keys: key (type: string)
+        mode: hash
         outputColumnNames: _col0, _col1
         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-        Group By Operator (GBY_5)
-          aggregations: count(_col1)
-          keys: _col0 (type: string)
-          mode: hash
-          outputColumnNames: _col0, _col1
+        Reduce Output Operator (RS_5)
+          key expressions: _col0 (type: string)
+          sort order: +
+          Map-reduce partition columns: _col0 (type: string)
           Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-          Reduce Output Operator (RS_6)
-            key expressions: _col0 (type: string)
-            sort order: +
-            Map-reduce partition columns: _col0 (type: string)
-            Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-            value expressions: _col1 (type: bigint)
-            Group By Operator (GBY_7)
-              aggregations: count(VALUE._col0)
-              keys: KEY._col0 (type: string)
-              mode: mergepartial
-              outputColumnNames: _col0, _col1
+          value expressions: _col1 (type: bigint)
+          Group By Operator (GBY_6)
+            aggregations: count(VALUE._col0)
+            keys: KEY._col0 (type: string)
+            mode: mergepartial
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+            Reduce Output Operator (RS_11)
+              key expressions: _col0 (type: string)
+              sort order: +
+              Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator (RS_12)
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
-                value expressions: _col1 (type: bigint)
-                Join Operator (JOIN_13)
-                  condition map:
-                       Inner Join 0 to 1
-                  keys:
-                    0 _col0 (type: string)
-                    1 _col0 (type: string)
-                  outputColumnNames: _col0, _col1, _col3
-                  Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: bigint)
+              Join Operator (JOIN_12)
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col3
+                Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
 

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/fetch_aggregation.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/fetch_aggregation.q.out b/ql/src/test/results/clientpositive/fetch_aggregation.q.out
index 121d6a4..91f47f8 100644
--- a/ql/src/test/results/clientpositive/fetch_aggregation.q.out
+++ b/ql/src/test/results/clientpositive/fetch_aggregation.q.out
@@ -17,10 +17,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0), sum(_col0), avg(_col0), min(_col0), max(_col0), std(_col0), variance(_col0)
+                aggregations: count(key), sum(key), avg(key), min(key), max(key), std(key), variance(key)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/gby_star.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/gby_star.q.out b/ql/src/test/results/clientpositive/gby_star.q.out
index a49f1bf..fb71835 100644
--- a/ql/src/test/results/clientpositive/gby_star.q.out
+++ b/ql/src/test/results/clientpositive/gby_star.q.out
@@ -17,11 +17,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: sum(key)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -93,22 +93,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (UDFToDouble(key) < 100.0) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
+              Group By Operator
+                aggregations: sum(key)
+                keys: key (type: string), value (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: sum(_col0)
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col2 (type: double)
+                  value expressions: _col2 (type: double)
       Reduce Operator Tree:
         Group By Operator
           aggregations: sum(VALUE._col0)
@@ -171,22 +167,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (UDFToDouble(key) < 100.0) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: sum(key)
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: sum(_col0)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: double)
+                  value expressions: _col1 (type: double)
       Reduce Operator Tree:
         Group By Operator
           aggregations: sum(VALUE._col0)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby12.q.out b/ql/src/test/results/clientpositive/groupby12.q.out
index 6eb3e96..b17da54 100644
--- a/ql/src/test/results/clientpositive/groupby12.q.out
+++ b/ql/src/test/results/clientpositive/groupby12.q.out
@@ -28,12 +28,12 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: string), _col1 (type: string)
+                key expressions: key (type: string), value (type: string)
                 sort order: ++
-                Map-reduce partition columns: _col0 (type: string)
+                Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby5_map.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby5_map.q.out b/ql/src/test/results/clientpositive/groupby5_map.q.out
index ea8ce4a..5fbd3d7 100644
--- a/ql/src/test/results/clientpositive/groupby5_map.q.out
+++ b/ql/src/test/results/clientpositive/groupby5_map.q.out
@@ -26,10 +26,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(key)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby5_map_skew.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby5_map_skew.q.out b/ql/src/test/results/clientpositive/groupby5_map_skew.q.out
index c0fced7..60b010b 100644
--- a/ql/src/test/results/clientpositive/groupby5_map_skew.q.out
+++ b/ql/src/test/results/clientpositive/groupby5_map_skew.q.out
@@ -26,10 +26,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(key)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_cube1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_cube1.q.out b/ql/src/test/results/clientpositive/groupby_cube1.q.out
index 62e9c54..f5a2d1c 100644
--- a/ql/src/test/results/clientpositive/groupby_cube1.q.out
+++ b/ql/src/test/results/clientpositive/groupby_cube1.q.out
@@ -206,11 +206,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), val (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, val
               Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: string), '0' (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT val)
+                keys: key (type: string), '0' (type: string), val (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 2 Data size: 60 Basic stats: COMPLETE Column stats: NONE
@@ -389,11 +389,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), val (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, val
               Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: string), '0' (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT val)
+                keys: key (type: string), '0' (type: string), val (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 2 Data size: 60 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_distinct_samekey.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_distinct_samekey.q.out b/ql/src/test/results/clientpositive/groupby_distinct_samekey.q.out
index bd92927..b7121d5 100644
--- a/ql/src/test/results/clientpositive/groupby_distinct_samekey.q.out
+++ b/ql/src/test/results/clientpositive/groupby_distinct_samekey.q.out
@@ -115,11 +115,11 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 120 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: int1 (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: int1
               Statistics: Num rows: 8 Data size: 120 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(DISTINCT _col0)
-                keys: _col0 (type: int)
+                aggregations: sum(DISTINCT int1)
+                keys: int1 (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 8 Data size: 120 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out b/ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out
index 6d44f32..6f9237d 100644
--- a/ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out
+++ b/ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out
@@ -36,11 +36,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: string), b (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: a, b
               Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: a (type: string), b (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
@@ -143,11 +143,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: string), b (type: string), c (type: string)
-              outputColumnNames: _col0, _col1, _col2
+              outputColumnNames: a, b, c
               Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col2)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: sum(c)
+                keys: a (type: string), b (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out b/ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out
index 5d1a7a8..d3e6bee 100644
--- a/ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out
+++ b/ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out
@@ -55,11 +55,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 72 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: string), b (type: string), c (type: string)
-              outputColumnNames: _col0, _col1, _col2
+              outputColumnNames: a, b, c
               Statistics: Num rows: 1 Data size: 72 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: avg(_col2), count()
-                keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
+                aggregations: avg(c), count()
+                keys: a (type: string), b (type: string), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 4 Data size: 288 Basic stats: COMPLETE Column stats: NONE
@@ -143,11 +143,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 72 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: string), b (type: string), c (type: string)
-              outputColumnNames: _col0, _col1, _col2
+              outputColumnNames: a, b, c
               Statistics: Num rows: 1 Data size: 72 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: avg(_col2), count()
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: avg(c), count()
+                keys: a (type: string), b (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 72 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out b/ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out
index bd7c2ff..e022297 100644
--- a/ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out
+++ b/ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out
@@ -42,10 +42,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: string), b (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: a, b
               Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: a (type: string), b (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
@@ -159,10 +159,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: string), b (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: a, b
               Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: a (type: string), b (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_grouping_sets6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_grouping_sets6.q.out b/ql/src/test/results/clientpositive/groupby_grouping_sets6.q.out
index d807d44..1d7b569 100644
--- a/ql/src/test/results/clientpositive/groupby_grouping_sets6.q.out
+++ b/ql/src/test/results/clientpositive/groupby_grouping_sets6.q.out
@@ -42,10 +42,10 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: '5' (type: string), b (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: a, b
                 Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
+                  keys: a (type: string), b (type: string), '0' (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 2 Data size: 72 Basic stats: COMPLETE Column stats: NONE
@@ -117,10 +117,10 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: '5' (type: string), b (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: a, b
                 Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
+                  keys: a (type: string), b (type: string), '0' (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 2 Data size: 72 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_position.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_position.q.out b/ql/src/test/results/clientpositive/groupby_position.q.out
index 7152101..3472134 100644
--- a/ql/src/test/results/clientpositive/groupby_position.q.out
+++ b/ql/src/test/results/clientpositive/groupby_position.q.out
@@ -563,20 +563,16 @@ STAGE PLANS:
             Filter Operator
               predicate: (((((UDFToDouble(key) > 10.0) and (UDFToDouble(key) < 20.0)) and (UDFToDouble(key) > 15.0)) and (UDFToDouble(key) < 25.0)) and key is not null) (type: boolean)
               Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                keys: key (type: string), value (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
@@ -653,20 +649,16 @@ STAGE PLANS:
             Filter Operator
               predicate: (((((UDFToDouble(key) > 15.0) and (UDFToDouble(key) < 25.0)) and (UDFToDouble(key) > 10.0)) and (UDFToDouble(key) < 20.0)) and key is not null) (type: boolean)
               Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                keys: key (type: string), value (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string)


[41/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
index 36f1099..7d9d99e 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
@@ -398,28 +398,28 @@ Stage-0
             compressed:false
             Statistics:Num rows: 10 Data size: 885 Basic stats: COMPLETE Column stats: COMPLETE
             table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
-            Select Operator [SEL_6]
+            Select Operator [SEL_5]
                outputColumnNames:["_col0","_col1","_col2"]
                Statistics:Num rows: 10 Data size: 885 Basic stats: COMPLETE Column stats: COMPLETE
-               Group By Operator [GBY_5]
+               Group By Operator [GBY_4]
                |  aggregations:["sum(VALUE._col0)"]
-               |  keys:KEY._col0 (type: float), KEY._col1 (type: int), KEY._col2 (type: string)
+               |  keys:KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float)
                |  outputColumnNames:["_col0","_col1","_col2","_col3"]
                |  Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 1 [SIMPLE_EDGE]
-                  Reduce Output Operator [RS_4]
-                     key expressions:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                     Map-reduce partition columns:_col0 (type: float), _col1 (type: int), _col2 (type: string)
+                  Reduce Output Operator [RS_3]
+                     key expressions:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                     Map-reduce partition columns:_col0 (type: string), _col1 (type: int), _col2 (type: float)
                      sort order:+++
                      Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
                      value expressions:_col3 (type: bigint)
-                     Group By Operator [GBY_3]
-                        aggregations:["sum(_col1)"]
-                        keys:_col0 (type: float), _col1 (type: int), _col2 (type: string)
+                     Group By Operator [GBY_2]
+                        aggregations:["sum(c_int)"]
+                        keys:key (type: string), c_int (type: int), c_float (type: float)
                         outputColumnNames:["_col0","_col1","_col2","_col3"]
                         Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator [SEL_1]
-                           outputColumnNames:["_col0","_col1","_col2"]
+                           outputColumnNames:["key","c_int","c_float"]
                            Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
                            TableScan [TS_0]
                               alias:cbo_t1
@@ -464,28 +464,28 @@ Stage-0
                         keys:_col0 (type: bigint), _col1 (type: float)
                         outputColumnNames:["_col0","_col1","_col2"]
                         Statistics:Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE
-                        Select Operator [SEL_6]
+                        Select Operator [SEL_5]
                            outputColumnNames:["_col0","_col1"]
                            Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
-                           Group By Operator [GBY_5]
+                           Group By Operator [GBY_4]
                            |  aggregations:["sum(VALUE._col0)"]
-                           |  keys:KEY._col0 (type: float), KEY._col1 (type: int), KEY._col2 (type: string)
+                           |  keys:KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float)
                            |  outputColumnNames:["_col0","_col1","_col2","_col3"]
                            |  Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
                            |<-Map 1 [SIMPLE_EDGE]
-                              Reduce Output Operator [RS_4]
-                                 key expressions:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                 Map-reduce partition columns:_col0 (type: float), _col1 (type: int), _col2 (type: string)
+                              Reduce Output Operator [RS_3]
+                                 key expressions:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                 Map-reduce partition columns:_col0 (type: string), _col1 (type: int), _col2 (type: float)
                                  sort order:+++
                                  Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
                                  value expressions:_col3 (type: bigint)
-                                 Group By Operator [GBY_3]
-                                    aggregations:["sum(_col1)"]
-                                    keys:_col0 (type: float), _col1 (type: int), _col2 (type: string)
+                                 Group By Operator [GBY_2]
+                                    aggregations:["sum(c_int)"]
+                                    keys:key (type: string), c_int (type: int), c_float (type: float)
                                     outputColumnNames:["_col0","_col1","_col2","_col3"]
                                     Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
                                     Select Operator [SEL_1]
-                                       outputColumnNames:["_col0","_col1","_col2"]
+                                       outputColumnNames:["key","c_int","c_float"]
                                        Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
                                        TableScan [TS_0]
                                           alias:cbo_t1
@@ -525,152 +525,149 @@ Stage-0
                   sort order:-+
                   Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions:_col0 (type: int), _col2 (type: bigint)
-                  Group By Operator [GBY_41]
-                  |  aggregations:["count(VALUE._col0)"]
-                  |  keys:KEY._col0 (type: int), KEY._col1 (type: bigint)
-                  |  outputColumnNames:["_col0","_col1","_col2"]
-                  |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                  |<-Reducer 5 [SIMPLE_EDGE]
-                     Reduce Output Operator [RS_40]
-                        key expressions:_col0 (type: int), _col1 (type: bigint)
-                        Map-reduce partition columns:_col0 (type: int), _col1 (type: bigint)
-                        sort order:++
-                        Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions:_col2 (type: bigint)
-                        Group By Operator [GBY_39]
-                           aggregations:["count()"]
-                           keys:_col0 (type: int), _col1 (type: bigint)
-                           outputColumnNames:["_col0","_col1","_col2"]
+                  Select Operator [SEL_41]
+                     outputColumnNames:["_col0","_col1","_col2"]
+                     Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                     Group By Operator [GBY_40]
+                     |  aggregations:["count(VALUE._col0)"]
+                     |  keys:KEY._col0 (type: bigint), KEY._col1 (type: int)
+                     |  outputColumnNames:["_col0","_col1","_col2"]
+                     |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                     |<-Reducer 5 [SIMPLE_EDGE]
+                        Reduce Output Operator [RS_39]
+                           key expressions:_col0 (type: bigint), _col1 (type: int)
+                           Map-reduce partition columns:_col0 (type: bigint), _col1 (type: int)
+                           sort order:++
                            Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                           Select Operator [SEL_37]
-                              outputColumnNames:["_col0","_col1"]
-                              Statistics:Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-                              Filter Operator [FIL_36]
-                                 predicate:((_col1 > 0) or (_col6 >= 0)) (type: boolean)
+                           value expressions:_col2 (type: bigint)
+                           Group By Operator [GBY_38]
+                              aggregations:["count()"]
+                              keys:_col2 (type: bigint), _col6 (type: int)
+                              outputColumnNames:["_col0","_col1","_col2"]
+                              Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                              Select Operator [SEL_37]
+                                 outputColumnNames:["_col2","_col6"]
                                  Statistics:Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-                                 Merge Join Operator [MERGEJOIN_55]
-                                 |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
-                                 |  outputColumnNames:["_col1","_col2","_col6"]
-                                 |  Statistics:Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
-                                 |<-Map 11 [SIMPLE_EDGE]
-                                 |  Reduce Output Operator [RS_34]
-                                 |     key expressions:_col0 (type: string)
-                                 |     Map-reduce partition columns:_col0 (type: string)
-                                 |     sort order:+
-                                 |     Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
-                                 |     value expressions:_col1 (type: int)
-                                 |     Select Operator [SEL_30]
-                                 |        outputColumnNames:["_col0","_col1"]
-                                 |        Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
-                                 |        Filter Operator [FIL_53]
-                                 |           predicate:key is not null (type: boolean)
-                                 |           Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
-                                 |           TableScan [TS_29]
-                                 |              alias:cbo_t3
-                                 |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-                                 |<-Reducer 4 [SIMPLE_EDGE]
-                                    Reduce Output Operator [RS_32]
-                                       key expressions:_col0 (type: string)
-                                       Map-reduce partition columns:_col0 (type: string)
-                                       sort order:+
-                                       Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                       value expressions:_col1 (type: int), _col2 (type: bigint)
-                                       Select Operator [SEL_28]
-                                          outputColumnNames:["_col0","_col1","_col2"]
+                                 Filter Operator [FIL_36]
+                                    predicate:((_col1 > 0) or (_col6 >= 0)) (type: boolean)
+                                    Statistics:Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                                    Merge Join Operator [MERGEJOIN_55]
+                                    |  condition map:[{"":"Inner Join 0 to 1"}]
+                                    |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
+                                    |  outputColumnNames:["_col1","_col2","_col6"]
+                                    |  Statistics:Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |<-Map 11 [SIMPLE_EDGE]
+                                    |  Reduce Output Operator [RS_34]
+                                    |     key expressions:_col0 (type: string)
+                                    |     Map-reduce partition columns:_col0 (type: string)
+                                    |     sort order:+
+                                    |     Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |     value expressions:_col1 (type: int)
+                                    |     Select Operator [SEL_30]
+                                    |        outputColumnNames:["_col0","_col1"]
+                                    |        Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |        Filter Operator [FIL_53]
+                                    |           predicate:key is not null (type: boolean)
+                                    |           Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |           TableScan [TS_29]
+                                    |              alias:cbo_t3
+                                    |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |<-Reducer 4 [SIMPLE_EDGE]
+                                       Reduce Output Operator [RS_32]
+                                          key expressions:_col0 (type: string)
+                                          Map-reduce partition columns:_col0 (type: string)
+                                          sort order:+
                                           Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                          Filter Operator [FIL_27]
-                                             predicate:((_col1 + _col4) >= 0) (type: boolean)
+                                          value expressions:_col1 (type: int), _col2 (type: bigint)
+                                          Select Operator [SEL_28]
+                                             outputColumnNames:["_col0","_col1","_col2"]
                                              Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                             Merge Join Operator [MERGEJOIN_54]
-                                             |  condition map:[{"":"Inner Join 0 to 1"}]
-                                             |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
-                                             |  outputColumnNames:["_col0","_col1","_col2","_col4"]
-                                             |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |<-Reducer 10 [SIMPLE_EDGE]
-                                             |  Reduce Output Operator [RS_25]
-                                             |     key expressions:_col0 (type: string)
-                                             |     Map-reduce partition columns:_col0 (type: string)
-                                             |     sort order:+
-                                             |     Statistics:Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |     value expressions:_col1 (type: int)
-                                             |     Select Operator [SEL_20]
-                                             |     |  outputColumnNames:["_col0","_col1"]
-                                             |     |  Statistics:Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |     |<-Reducer 9 [SIMPLE_EDGE]
-                                             |        Reduce Output Operator [RS_19]
-                                             |           key expressions:_col3 (type: double), _col2 (type: bigint)
-                                             |           sort order:-+
-                                             |           Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |           value expressions:_col0 (type: string), _col1 (type: int)
-                                             |           Select Operator [SEL_18]
-                                             |              outputColumnNames:["_col0","_col1","_col2","_col3"]
-                                             |              Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |              Group By Operator [GBY_17]
-                                             |              |  aggregations:["sum(VALUE._col0)"]
-                                             |              |  keys:KEY._col0 (type: float), KEY._col1 (type: int), KEY._col2 (type: string)
-                                             |              |  outputColumnNames:["_col0","_col1","_col2","_col3"]
-                                             |              |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |              |<-Map 8 [SIMPLE_EDGE]
-                                             |                 Reduce Output Operator [RS_16]
-                                             |                    key expressions:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                             |                    Map-reduce partition columns:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                             |                    sort order:+++
-                                             |                    Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |                    value expressions:_col3 (type: bigint)
-                                             |                    Group By Operator [GBY_15]
-                                             |                       aggregations:["sum(_col1)"]
-                                             |                       keys:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                             |                       outputColumnNames:["_col0","_col1","_col2","_col3"]
-                                             |                       Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |                       Select Operator [SEL_13]
-                                             |                          outputColumnNames:["_col0","_col1","_col2"]
-                                             |                          Statistics:Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |                          Filter Operator [FIL_52]
-                                             |                             predicate:((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and key is not null) (type: boolean)
-                                             |                             Statistics:Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |                             TableScan [TS_11]
-                                             |                                alias:cbo_t2
-                                             |                                Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |<-Reducer 3 [SIMPLE_EDGE]
-                                                Reduce Output Operator [RS_23]
-                                                   key expressions:_col0 (type: string)
-                                                   Map-reduce partition columns:_col0 (type: string)
-                                                   sort order:+
-                                                   Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
-                                                   value expressions:_col1 (type: int), _col2 (type: bigint)
-                                                   Select Operator [SEL_9]
-                                                   |  outputColumnNames:["_col0","_col1","_col2"]
-                                                   |  Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
-                                                   |<-Reducer 2 [SIMPLE_EDGE]
-                                                      Reduce Output Operator [RS_8]
-                                                         key expressions:_col0 (type: string)
-                                                         sort order:+
-                                                         Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
-                                                         value expressions:_col1 (type: int), _col2 (type: bigint)
-                                                         Select Operator [SEL_7]
-                                                            outputColumnNames:["_col0","_col1","_col2"]
+                                             Filter Operator [FIL_27]
+                                                predicate:((_col1 + _col4) >= 0) (type: boolean)
+                                                Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                Merge Join Operator [MERGEJOIN_54]
+                                                |  condition map:[{"":"Inner Join 0 to 1"}]
+                                                |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
+                                                |  outputColumnNames:["_col0","_col1","_col2","_col4"]
+                                                |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |<-Reducer 10 [SIMPLE_EDGE]
+                                                |  Reduce Output Operator [RS_25]
+                                                |     key expressions:_col0 (type: string)
+                                                |     Map-reduce partition columns:_col0 (type: string)
+                                                |     sort order:+
+                                                |     Statistics:Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |     value expressions:_col1 (type: int)
+                                                |     Select Operator [SEL_20]
+                                                |     |  outputColumnNames:["_col0","_col1"]
+                                                |     |  Statistics:Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |     |<-Reducer 9 [SIMPLE_EDGE]
+                                                |        Reduce Output Operator [RS_19]
+                                                |           key expressions:_col3 (type: double), _col2 (type: bigint)
+                                                |           sort order:-+
+                                                |           Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |           value expressions:_col0 (type: string), _col1 (type: int)
+                                                |           Select Operator [SEL_17]
+                                                |              outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                                |              Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |              Group By Operator [GBY_16]
+                                                |              |  aggregations:["sum(VALUE._col0)"]
+                                                |              |  keys:KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float)
+                                                |              |  outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                                |              |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |              |<-Map 8 [SIMPLE_EDGE]
+                                                |                 Reduce Output Operator [RS_15]
+                                                |                    key expressions:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                                |                    Map-reduce partition columns:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                                |                    sort order:+++
+                                                |                    Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |                    value expressions:_col3 (type: bigint)
+                                                |                    Group By Operator [GBY_14]
+                                                |                       aggregations:["sum(c_int)"]
+                                                |                       keys:key (type: string), c_int (type: int), c_float (type: float)
+                                                |                       outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                                |                       Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |                       Filter Operator [FIL_52]
+                                                |                          predicate:((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and key is not null) (type: boolean)
+                                                |                          Statistics:Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |                          TableScan [TS_11]
+                                                |                             alias:cbo_t2
+                                                |                             Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |<-Reducer 3 [SIMPLE_EDGE]
+                                                   Reduce Output Operator [RS_23]
+                                                      key expressions:_col0 (type: string)
+                                                      Map-reduce partition columns:_col0 (type: string)
+                                                      sort order:+
+                                                      Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+                                                      value expressions:_col1 (type: int), _col2 (type: bigint)
+                                                      Select Operator [SEL_9]
+                                                      |  outputColumnNames:["_col0","_col1","_col2"]
+                                                      |  Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+                                                      |<-Reducer 2 [SIMPLE_EDGE]
+                                                         Reduce Output Operator [RS_8]
+                                                            key expressions:_col0 (type: string)
+                                                            sort order:+
                                                             Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
-                                                            Group By Operator [GBY_6]
-                                                            |  aggregations:["sum(VALUE._col0)"]
-                                                            |  keys:KEY._col0 (type: float), KEY._col1 (type: int), KEY._col2 (type: string)
-                                                            |  outputColumnNames:["_col0","_col1","_col2","_col3"]
-                                                            |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                                            |<-Map 1 [SIMPLE_EDGE]
-                                                               Reduce Output Operator [RS_5]
-                                                                  key expressions:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                                                  Map-reduce partition columns:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                                                  sort order:+++
-                                                                  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                                                  value expressions:_col3 (type: bigint)
-                                                                  Group By Operator [GBY_4]
-                                                                     aggregations:["sum(_col1)"]
-                                                                     keys:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                                                     outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                                            value expressions:_col1 (type: int), _col2 (type: bigint)
+                                                            Select Operator [SEL_6]
+                                                               outputColumnNames:["_col0","_col1","_col2"]
+                                                               Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+                                                               Group By Operator [GBY_5]
+                                                               |  aggregations:["sum(VALUE._col0)"]
+                                                               |  keys:KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float)
+                                                               |  outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                                               |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                               |<-Map 1 [SIMPLE_EDGE]
+                                                                  Reduce Output Operator [RS_4]
+                                                                     key expressions:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                                                     Map-reduce partition columns:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                                                     sort order:+++
                                                                      Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                                                     Select Operator [SEL_2]
-                                                                        outputColumnNames:["_col0","_col1","_col2"]
-                                                                        Statistics:Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
+                                                                     value expressions:_col3 (type: bigint)
+                                                                     Group By Operator [GBY_3]
+                                                                        aggregations:["sum(c_int)"]
+                                                                        keys:key (type: string), c_int (type: int), c_float (type: float)
+                                                                        outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                                                        Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                                                                         Filter Operator [FIL_51]
                                                                            predicate:((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and key is not null) (type: boolean)
                                                                            Statistics:Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
@@ -711,143 +708,146 @@ Stage-0
                   sort order:+-
                   Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions:_col1 (type: bigint), _col2 (type: bigint)
-                  Group By Operator [GBY_40]
-                  |  aggregations:["count(VALUE._col0)"]
-                  |  keys:KEY._col0 (type: int), KEY._col1 (type: bigint)
-                  |  outputColumnNames:["_col0","_col1","_col2"]
-                  |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                  |<-Reducer 5 [SIMPLE_EDGE]
-                     Reduce Output Operator [RS_39]
-                        key expressions:_col0 (type: int), _col1 (type: bigint)
-                        Map-reduce partition columns:_col0 (type: int), _col1 (type: bigint)
-                        sort order:++
-                        Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions:_col2 (type: bigint)
-                        Group By Operator [GBY_38]
-                           aggregations:["count()"]
-                           keys:_col0 (type: int), _col1 (type: bigint)
-                           outputColumnNames:["_col0","_col1","_col2"]
+                  Select Operator [SEL_40]
+                     outputColumnNames:["_col0","_col1","_col2"]
+                     Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                     Group By Operator [GBY_39]
+                     |  aggregations:["count(VALUE._col0)"]
+                     |  keys:KEY._col0 (type: bigint), KEY._col1 (type: int)
+                     |  outputColumnNames:["_col0","_col1","_col2"]
+                     |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                     |<-Reducer 5 [SIMPLE_EDGE]
+                        Reduce Output Operator [RS_38]
+                           key expressions:_col0 (type: bigint), _col1 (type: int)
+                           Map-reduce partition columns:_col0 (type: bigint), _col1 (type: int)
+                           sort order:++
                            Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                           Select Operator [SEL_34]
-                              outputColumnNames:["_col0","_col1"]
-                              Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                              Filter Operator [FIL_48]
-                                 predicate:((((_col6 > 0) and ((_col6 >= 1) or (_col2 >= 1))) and ((UDFToLong(_col6) + _col2) >= 0)) and ((_col1 > 0) or (_col6 >= 0))) (type: boolean)
+                           value expressions:_col2 (type: bigint)
+                           Group By Operator [GBY_37]
+                              aggregations:["count()"]
+                              keys:_col2 (type: bigint), _col6 (type: int)
+                              outputColumnNames:["_col0","_col1","_col2"]
+                              Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                              Select Operator [SEL_34]
+                                 outputColumnNames:["_col2","_col6"]
                                  Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                                 Merge Join Operator [MERGEJOIN_53]
-                                 |  condition map:[{"":"Left Outer Join0 to 1"}]
-                                 |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
-                                 |  outputColumnNames:["_col1","_col2","_col6"]
-                                 |  Statistics:Num rows: 4 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                                 |<-Map 10 [SIMPLE_EDGE]
-                                 |  Reduce Output Operator [RS_31]
-                                 |     key expressions:_col0 (type: string)
-                                 |     Map-reduce partition columns:_col0 (type: string)
-                                 |     sort order:+
-                                 |     Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-                                 |     value expressions:_col1 (type: int)
-                                 |     Select Operator [SEL_29]
-                                 |        outputColumnNames:["_col0","_col1"]
-                                 |        Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-                                 |        TableScan [TS_28]
-                                 |           alias:cbo_t3
-                                 |           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-                                 |<-Reducer 4 [SIMPLE_EDGE]
-                                    Reduce Output Operator [RS_30]
-                                       key expressions:_col0 (type: string)
-                                       Map-reduce partition columns:_col0 (type: string)
-                                       sort order:+
-                                       Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                       value expressions:_col1 (type: int), _col2 (type: bigint)
-                                       Select Operator [SEL_27]
-                                          outputColumnNames:["_col0","_col1","_col2"]
+                                 Filter Operator [FIL_48]
+                                    predicate:((((_col6 > 0) and ((_col6 >= 1) or (_col2 >= 1))) and ((UDFToLong(_col6) + _col2) >= 0)) and ((_col1 > 0) or (_col6 >= 0))) (type: boolean)
+                                    Statistics:Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                                    Merge Join Operator [MERGEJOIN_53]
+                                    |  condition map:[{"":"Left Outer Join0 to 1"}]
+                                    |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
+                                    |  outputColumnNames:["_col1","_col2","_col6"]
+                                    |  Statistics:Num rows: 4 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |<-Map 10 [SIMPLE_EDGE]
+                                    |  Reduce Output Operator [RS_31]
+                                    |     key expressions:_col0 (type: string)
+                                    |     Map-reduce partition columns:_col0 (type: string)
+                                    |     sort order:+
+                                    |     Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |     value expressions:_col1 (type: int)
+                                    |     Select Operator [SEL_29]
+                                    |        outputColumnNames:["_col0","_col1"]
+                                    |        Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |        TableScan [TS_28]
+                                    |           alias:cbo_t3
+                                    |           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |<-Reducer 4 [SIMPLE_EDGE]
+                                       Reduce Output Operator [RS_30]
+                                          key expressions:_col0 (type: string)
+                                          Map-reduce partition columns:_col0 (type: string)
+                                          sort order:+
                                           Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                          Filter Operator [FIL_26]
-                                             predicate:((_col1 + _col4) >= 0) (type: boolean)
+                                          value expressions:_col1 (type: int), _col2 (type: bigint)
+                                          Select Operator [SEL_27]
+                                             outputColumnNames:["_col0","_col1","_col2"]
                                              Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                             Merge Join Operator [MERGEJOIN_52]
-                                             |  condition map:[{"":"Left Outer Join0 to 1"}]
-                                             |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
-                                             |  outputColumnNames:["_col0","_col1","_col2","_col4"]
-                                             |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |<-Reducer 3 [SIMPLE_EDGE]
-                                             |  Reduce Output Operator [RS_23]
-                                             |     key expressions:_col0 (type: string)
-                                             |     Map-reduce partition columns:_col0 (type: string)
-                                             |     sort order:+
-                                             |     Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |     value expressions:_col1 (type: int), _col2 (type: bigint)
-                                             |     Select Operator [SEL_11]
-                                             |     |  outputColumnNames:["_col0","_col1","_col2"]
-                                             |     |  Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |     |<-Reducer 2 [SIMPLE_EDGE]
-                                             |        Reduce Output Operator [RS_10]
-                                             |           key expressions:_col3 (type: bigint), _col1 (type: int)
-                                             |           sort order:+-
-                                             |           Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |           value expressions:_col0 (type: string), _col2 (type: bigint)
-                                             |           Select Operator [SEL_9]
-                                             |              outputColumnNames:["_col0","_col1","_col2","_col3"]
-                                             |              Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |              Group By Operator [GBY_8]
-                                             |              |  aggregations:["sum(VALUE._col0)"]
-                                             |              |  keys:KEY._col0 (type: float), KEY._col1 (type: int), KEY._col2 (type: string)
-                                             |              |  outputColumnNames:["_col0","_col1","_col2","_col3"]
-                                             |              |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |              |<-Map 1 [SIMPLE_EDGE]
-                                             |                 Reduce Output Operator [RS_7]
-                                             |                    key expressions:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                             |                    Map-reduce partition columns:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                             |                    sort order:+++
-                                             |                    Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |                    value expressions:_col3 (type: bigint)
-                                             |                    Group By Operator [GBY_6]
-                                             |                       aggregations:["sum(_col1)"]
-                                             |                       keys:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                             |                       outputColumnNames:["_col0","_col1","_col2","_col3"]
-                                             |                       Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |                       Select Operator [SEL_2]
-                                             |                          outputColumnNames:["_col0","_col1","_col2"]
-                                             |                          Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |                          Filter Operator [FIL_49]
-                                             |                             predicate:((((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and (c_float > 0.0)) and ((c_int >= 1) or (c_float >= 1.0))) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean)
-                                             |                             Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |                             TableScan [TS_0]
-                                             |                                alias:cbo_t1
-                                             |                                Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-                                             |<-Reducer 9 [SIMPLE_EDGE]
-                                                Reduce Output Operator [RS_24]
-                                                   key expressions:_col0 (type: string)
-                                                   Map-reduce partition columns:_col0 (type: string)
-                                                   sort order:+
-                                                   Statistics:Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
-                                                   value expressions:_col1 (type: int)
-                                                   Select Operator [SEL_22]
-                                                      outputColumnNames:["_col0","_col1"]
+                                             Filter Operator [FIL_26]
+                                                predicate:((_col1 + _col4) >= 0) (type: boolean)
+                                                Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                Merge Join Operator [MERGEJOIN_52]
+                                                |  condition map:[{"":"Left Outer Join0 to 1"}]
+                                                |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
+                                                |  outputColumnNames:["_col0","_col1","_col2","_col4"]
+                                                |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |<-Reducer 3 [SIMPLE_EDGE]
+                                                |  Reduce Output Operator [RS_23]
+                                                |     key expressions:_col0 (type: string)
+                                                |     Map-reduce partition columns:_col0 (type: string)
+                                                |     sort order:+
+                                                |     Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |     value expressions:_col1 (type: int), _col2 (type: bigint)
+                                                |     Select Operator [SEL_11]
+                                                |     |  outputColumnNames:["_col0","_col1","_col2"]
+                                                |     |  Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |     |<-Reducer 2 [SIMPLE_EDGE]
+                                                |        Reduce Output Operator [RS_10]
+                                                |           key expressions:_col3 (type: bigint), _col1 (type: int)
+                                                |           sort order:+-
+                                                |           Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |           value expressions:_col0 (type: string), _col2 (type: bigint)
+                                                |           Select Operator [SEL_8]
+                                                |              outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                                |              Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |              Group By Operator [GBY_7]
+                                                |              |  aggregations:["sum(VALUE._col0)"]
+                                                |              |  keys:KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float)
+                                                |              |  outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                                |              |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |              |<-Map 1 [SIMPLE_EDGE]
+                                                |                 Reduce Output Operator [RS_6]
+                                                |                    key expressions:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                                |                    Map-reduce partition columns:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                                |                    sort order:+++
+                                                |                    Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |                    value expressions:_col3 (type: bigint)
+                                                |                    Group By Operator [GBY_5]
+                                                |                       aggregations:["sum(_col2)"]
+                                                |                       keys:_col0 (type: string), _col2 (type: int), _col3 (type: float)
+                                                |                       outputColumnNames:["_col0","_col1","_col2","_col3"]
+                                                |                       Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |                       Select Operator [SEL_2]
+                                                |                          outputColumnNames:["_col0","_col2","_col3"]
+                                                |                          Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |                          Filter Operator [FIL_49]
+                                                |                             predicate:((((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and (c_float > 0.0)) and ((c_int >= 1) or (c_float >= 1.0))) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean)
+                                                |                             Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |                             TableScan [TS_0]
+                                                |                                alias:cbo_t1
+                                                |                                Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |<-Reducer 9 [SIMPLE_EDGE]
+                                                   Reduce Output Operator [RS_24]
+                                                      key expressions:_col0 (type: string)
+                                                      Map-reduce partition columns:_col0 (type: string)
+                                                      sort order:+
                                                       Statistics:Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
-                                                      Group By Operator [GBY_21]
-                                                      |  keys:KEY._col0 (type: float), KEY._col1 (type: int), KEY._col2 (type: string)
-                                                      |  outputColumnNames:["_col0","_col1","_col2"]
-                                                      |  Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                                      |<-Map 8 [SIMPLE_EDGE]
-                                                         Reduce Output Operator [RS_20]
-                                                            key expressions:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                                            Map-reduce partition columns:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                                            sort order:+++
-                                                            Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                                            Group By Operator [GBY_19]
-                                                               keys:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                                               outputColumnNames:["_col0","_col1","_col2"]
+                                                      value expressions:_col1 (type: int)
+                                                      Select Operator [SEL_21]
+                                                         outputColumnNames:["_col0","_col1"]
+                                                         Statistics:Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
+                                                         Group By Operator [GBY_20]
+                                                         |  keys:KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float)
+                                                         |  outputColumnNames:["_col0","_col1","_col2"]
+                                                         |  Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                                                         |<-Map 8 [SIMPLE_EDGE]
+                                                            Reduce Output Operator [RS_19]
+                                                               key expressions:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                                               Map-reduce partition columns:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                                               sort order:+++
                                                                Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                                               Select Operator [SEL_15]
+                                                               Group By Operator [GBY_18]
+                                                                  keys:_col0 (type: string), _col2 (type: int), _col3 (type: float)
                                                                   outputColumnNames:["_col0","_col1","_col2"]
                                                                   Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                                                  Filter Operator [FIL_50]
-                                                                     predicate:((((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and (c_float > 0.0)) and ((c_int >= 1) or (c_float >= 1.0))) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean)
+                                                                  Select Operator [SEL_15]
+                                                                     outputColumnNames:["_col0","_col2","_col3"]
                                                                      Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                                                     TableScan [TS_13]
-                                                                        alias:cbo_t2
-                                                                        Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                                     Filter Operator [FIL_50]
+                                                                        predicate:((((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and (c_float > 0.0)) and ((c_int >= 1) or (c_float >= 1.0))) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean)
+                                                                        Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                                                                        TableScan [TS_13]
+                                                                           alias:cbo_t2
+                                                                           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)  group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) cbo_t1 right outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 2) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c
 PREHOOK: type: QUERY
@@ -867,129 +867,132 @@ Stage-0
       limit:-1
       Stage-1
          Reducer 5
-         File Output Operator [FS_36]
+         File Output Operator [FS_35]
             compressed:false
             Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
             table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
-            Group By Operator [GBY_34]
-            |  aggregations:["count(VALUE._col0)"]
-            |  keys:KEY._col0 (type: int), KEY._col1 (type: bigint)
-            |  outputColumnNames:["_col0","_col1","_col2"]
-            |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-            |<-Reducer 4 [SIMPLE_EDGE]
-               Reduce Output Operator [RS_33]
-                  key expressions:_col0 (type: int), _col1 (type: bigint)
-                  Map-reduce partition columns:_col0 (type: int), _col1 (type: bigint)
-                  sort order:++
-                  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions:_col2 (type: bigint)
-                  Group By Operator [GBY_32]
-                     aggregations:["count()"]
-                     keys:_col0 (type: int), _col1 (type: bigint)
-                     outputColumnNames:["_col0","_col1","_col2"]
+            Select Operator [SEL_34]
+               outputColumnNames:["_col0","_col1","_col2"]
+               Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+               Group By Operator [GBY_33]
+               |  aggregations:["count(VALUE._col0)"]
+               |  keys:KEY._col0 (type: bigint), KEY._col1 (type: int)
+               |  outputColumnNames:["_col0","_col1","_col2"]
+               |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+               |<-Reducer 4 [SIMPLE_EDGE]
+                  Reduce Output Operator [RS_32]
+                     key expressions:_col0 (type: bigint), _col1 (type: int)
+                     Map-reduce partition columns:_col0 (type: bigint), _col1 (type: int)
+                     sort order:++
                      Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                     Select Operator [SEL_30]
-                        outputColumnNames:["_col0","_col1"]
+                     value expressions:_col2 (type: bigint)
+                     Group By Operator [GBY_31]
+                        aggregations:["count()"]
+                        keys:_col2 (type: bigint), _col6 (type: int)
+                        outputColumnNames:["_col0","_col1","_col2"]
                         Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                        Filter Operator [FIL_29]
-                           predicate:(((_col1 + _col4) >= 2) and ((_col1 > 0) or (_col6 >= 0))) (type: boolean)
+                        Select Operator [SEL_30]
+                           outputColumnNames:["_col2","_col6"]
                            Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                           Merge Join Operator [MERGEJOIN_41]
-                           |  condition map:[{"":"Right Outer Join0 to 1"},{"":"Right Outer Join0 to 2"}]
-                           |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)","2":"_col0 (type: string)"}
-                           |  outputColumnNames:["_col1","_col2","_col4","_col6"]
-                           |  Statistics:Num rows: 4 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
-                           |<-Map 8 [SIMPLE_EDGE]
-                           |  Reduce Output Operator [RS_27]
-                           |     key expressions:_col0 (type: string)
-                           |     Map-reduce partition columns:_col0 (type: string)
-                           |     sort order:+
-                           |     Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-                           |     value expressions:_col1 (type: int)
-                           |     Select Operator [SEL_24]
-                           |        outputColumnNames:["_col0","_col1"]
-                           |        Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-                           |        TableScan [TS_23]
-                           |           alias:cbo_t3
-                           |           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-                           |<-Reducer 3 [SIMPLE_EDGE]
-                           |  Reduce Output Operator [RS_25]
-                           |     key expressions:_col0 (type: string)
-                           |     Map-reduce partition columns:_col0 (type: string)
-                           |     sort order:+
-                           |     Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
-                           |     value expressions:_col1 (type: int), _col2 (type: bigint)
-                           |     Select Operator [SEL_11]
-                           |     |  outputColumnNames:["_col0","_col1","_col2"]
-                           |     |  Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
-                           |     |<-Reducer 2 [SIMPLE_EDGE]
-                           |        Reduce Output Operator [RS_10]
-                           |           key expressions:_col3 (type: bigint), _col0 (type: string)
-                           |           sort order:+-
-                           |           Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
-                           |           value expressions:_col1 (type: int), _col2 (type: bigint)
-                           |           Select Operator [SEL_9]
-                           |              outputColumnNames:["_col0","_col1","_col2","_col3"]
-                           |              Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
-                           |              Group By Operator [GBY_8]
-                           |              |  aggregations:["sum(VALUE._col0)"]
-                           |              |  keys:KEY._col0 (type: float), KEY._col1 (type: int), KEY._col2 (type: string)
-                           |              |  outputColumnNames:["_col0","_col1","_col2","_col3"]
-                           |              |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                           |              |<-Map 1 [SIMPLE_EDGE]
-                           |                 Reduce Output Operator [RS_7]
-                           |                    key expressions:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                           |                    Map-reduce partition columns:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                           |                    sort order:+++
-                           |                    Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                           |                    value expressions:_col3 (type: bigint)
-                           |                    Group By Operator [GBY_6]
-                           |                       aggregations:["sum(_col1)"]
-                           |                       keys:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                           |                       outputColumnNames:["_col0","_col1","_col2","_col3"]
-                           |                       Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
-                           |                       Select Operator [SEL_2]
-                           |                          outputColumnNames:["_col0","_col1","_col2"]
-                           |                          Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                           |                          Filter Operator [FIL_39]
-                           |                             predicate:((((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and (c_float > 0.0)) and ((c_int >= 1) or (c_float >= 1.0))) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean)
-                           |                             Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                           |                             TableScan [TS_0]
-                           |                                alias:cbo_t1
-                           |                                Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-                           |<-Reducer 7 [SIMPLE_EDGE]
-                              Reduce Output Operator [RS_26]
-                                 key expressions:_col0 (type: string)
-                                 Map-reduce partition columns:_col0 (type: string)
-                                 sort order:+
-                                 Statistics:Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
-                                 value expressions:_col1 (type: int)
-                                 Select Operator [SEL_22]
-                                    outputColumnNames:["_col0","_col1"]
+                           Filter Operator [FIL_29]
+                              predicate:(((_col1 + _col4) >= 2) and ((_col1 > 0) or (_col6 >= 0))) (type: boolean)
+                              Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                              Merge Join Operator [MERGEJOIN_40]
+                              |  condition map:[{"":"Right Outer Join0 to 1"},{"":"Right Outer Join0 to 2"}]
+                              |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)","2":"_col0 (type: string)"}
+                              |  outputColumnNames:["_col1","_col2","_col4","_col6"]
+                              |  Statistics:Num rows: 4 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+                              |<-Map 8 [SIMPLE_EDGE]
+                              |  Reduce Output Operator [RS_27]
+                              |     key expressions:_col0 (type: string)
+                              |     Map-reduce partition columns:_col0 (type: string)
+                              |     sort order:+
+                              |     Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
+                              |     value expressions:_col1 (type: int)
+                              |     Select Operator [SEL_24]
+                              |        outputColumnNames:["_col0","_col1"]
+                              |        Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
+                              |        TableScan [TS_23]
+                              |           alias:cbo_t3
+                              |           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                              |<-Reducer 3 [SIMPLE_EDGE]
+                              |  Reduce Output Operator [RS_25]
+                              |     key expressions:_col0 (type: string)
+                              |     Map-reduce partition columns:_col0 (type: string)
+                              |     sort order:+
+                              |     Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+                              |     value expressions:_col1 (type: int), _col2 (type: bigint)
+                              |     Select Operator [SEL_11]
+                              |     |  outputColumnNames:["_col0","_col1","_col2"]
+                              |     |  Statistics:Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+                              |     |<-Reducer 2 [SIMPLE_EDGE]
+                              |        Reduce Output Operator [RS_10]
+                              |           key expressions:_col3 (type: bigint), _col0 (type: string)
+                              |           sort order:+-
+                              |           Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                              |           value expressions:_col1 (type: int), _col2 (type: bigint)
+                              |           Select Operator [SEL_8]
+                              |              outputColumnNames:["_col0","_col1","_col2","_col3"]
+                              |              Statistics:Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                              |              Group By Operator [GBY_7]
+                              |              |  aggregations:["sum(VALUE._col0)"]
+                              |              |  keys:KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float)
+                              |              |  outputColumnNames:["_col0","_col1","_col2","_col3"]
+                              |              |  Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                              |              |<-Map 1 [SIMPLE_EDGE]
+                              |                 Reduce Output Operator [RS_6]
+                              |                    key expressions:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                              |                    Map-reduce partition columns:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                              |                    sort order:+++
+                              |                    Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                              |                    value expressions:_col3 (type: bigint)
+                              |                    Group By Operator [GBY_5]
+                              |                       aggregations:["sum(_col2)"]
+                              |                       keys:_col0 (type: string), _col2 (type: int), _col3 (type: float)
+                              |                       outputColumnNames:["_col0","_col1","_col2","_col3"]
+                              |                       Statistics:Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                              |                       Select Operator [SEL_2]
+                              |                          outputColumnNames:["_col0","_col2","_col3"]
+                              |                          Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                              |                          Filter Operator [FIL_38]
+                              |                             predicate:((((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and (c_float > 0.0)) and ((c_int >= 1) or (c_float >= 1.0))) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean)
+                              |                             Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                              |                             TableScan [TS_0]
+                              |                                alias:cbo_t1
+                              |                                Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                              |<-Reducer 7 [SIMPLE_EDGE]
+                                 Reduce Output Operator [RS_26]
+                                    key expressions:_col0 (type: string)
+                                    Map-reduce partition columns:_col0 (type: string)
+                                    sort order:+
                                     Statistics:Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
-                                    Group By Operator [GBY_21]
-                                    |  keys:KEY._col0 (type: float), KEY._col1 (type: int), KEY._col2 (type: string)
-                                    |  outputColumnNames:["_col0","_col1","_col2"]
-                                    |  Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                    |<-Map 6 [SIMPLE_EDGE]
-                                       Reduce Output Operator [RS_20]
-                                          key expressions:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                          Map-reduce partition columns:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                          sort order:+++
-                                          Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                          Group By Operator [GBY_19]
-                                             keys:_col0 (type: float), _col1 (type: int), _col2 (type: string)
-                                             outputColumnNames:["_col0","_col1","_col2"]
+                                    value expressions:_col1 (type: int)
+                                    Select Operator [SEL_21]
+                                       outputColumnNames:["_col0","_col1"]
+                                       Statistics:Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
+                                       Group By Operator [GBY_20]
+                                       |  keys:KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float)
+                                       |  outputColumnNames:["_col0","_col1","_col2"]
+                                       |  Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                                       |<-Map 6 [SIMPLE_EDGE]
+                                          Reduce Output Operator [RS_19]
+                                             key expressions:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                             Map-reduce partition columns:_col0 (type: string), _col1 (type: int), _col2 (type: float)
+                                             sort order:+++
                                              Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                             Select Operator [SEL_15]
+                                             Group By Operator [GBY_18]
+                                                keys:_col0 (type: string), _col2 (type: int), _col3 (type: float)
                                                 outputColumnNames:["_col0","_col1","_col2"]
                                                 Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                                Filter Operator [FIL_40]
-                                                   predicate:((((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and (c_float > 0.0)) and ((c_int >= 1) or (c_float >= 1.0))) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean)
+                                                Select Operator [SEL_15]
+                                                   outputColumnNames:["_col0","_col2","_col3"]
                                                    Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
-                                                   TableScan [TS_13]
-                                                      alias:cbo_t2
-                                                      Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                   Filter Operator [FIL_39]
+                                                      predicate:((((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and (c_float > 0.0)) and ((c_int >= 1) or (c_float >= 1.0))) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean)
+                                                      Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                                                      TableScan [TS_13]
+                                                         alias:cbo_t2
+                                                         Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)  group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) cbo_t1 full outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by p+q desc, r asc) cbo_t2 on cbo_t1.a=p full outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int
 PREHOOK: type: QUERY
@@ -1011,150 +1014,153 @@ Stage-0
       limit:-1
       Stage-1
          Reducer 6
-         File Output Operator [FS_43]
+         File Output Operator [FS_42]
             compressed:false
             Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
             table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
-            Select Operator [SEL_42]
+            Select Operator [SEL_41]
             |  outputColumnNames:["_col0","_col1","_col2"]
             |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Reducer 5 [SIMPLE_EDGE]
-               Reduce Output Operator [RS_41]
+               Reduce Output Operator [RS_40]
                   key expressions:_col0 (type: int)
                   sort order:+
                   Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions:_col1 (type: bigint), _col2 (type: bigint)
-                  Group By Operator [GBY_39]
-                  |  aggregations:["count(VALUE._col0)"]
-                  |  keys:KEY._col0 (type: int), KEY._col1 (type: bigint)
-                  |  outputColumnNames:["_col0","_col1","_col2"]
-                  |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                  |<-Reducer 4 [SIMPLE_EDGE]
-                     Reduce Output Operator [RS_38]
-                        key expressions:_col0 (type: int), _col1 (type: bigint)
-                        Map-reduce partition columns:_col0 (type: int), _col1 (type: bigint)
-                        sort order:++
-                        Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions:_col2 (type: bigint)
-                        Group By Operator [GBY_37]
-                           aggregations:["count()"]
-                           keys:_col0 (type: int), _col1 (type: bigint)
-                           outputColumnNames:["_col0","_col1","_col2"]
+                  Select Operator [SEL_39]
+                     outputColumnNames:["_col0","_col1","_col2"]
+                     Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                     Group By Operator [GBY_38]
+                     |  aggregations:["count(VALUE._col0)"]
+                     |  keys:KEY._col0 (type: bigint), KEY._col1 (type: int)
+                     |  outputColumnNames:["_col0","_col1","_col2"]
+                     |  Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                     |<-Reducer 4 [SIMPLE_EDGE]
+                        Reduce Output Operator [RS_37]
+                           key expressions:_col0 (type: bigint), _col1 (type: int)
+                           Map-reduce partition columns:_col0 (type: bigint), _col1 (type: int)
+                           sort order:++
                            Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                           Select Operator [SEL_33]
-                              outputColumnNames:["_col0","_col1"]
+                           value expressions:_col2 (type: bigint)
+                           Group By Operator [GBY_36]
+                              aggregations:["count()"]
+                              keys:_col2 (type: bigint), _col6 (type: int)
+                              outputColumnNames:["_col0","_col1","_col2"]
                               Statistics:Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                              Filter Operator [FIL_45]
-                                 predicate:(((((_col6 > 0) and ((_col6 >= 1) or (_col2 >= 1))) and ((UDFToLong(_col6)

<TRUNCATED>

[08/50] [abbrv] hive git commit: HIVE-11727: Hive on Tez through Oozie: Some queries fail with fnf exception (Gunther Hagleitner, reviewed by Vikram Dixit K)

Posted by xu...@apache.org.
HIVE-11727: Hive on Tez through Oozie: Some queries fail with fnf exception (Gunther Hagleitner, reviewed by Vikram Dixit K)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/594e25a2
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/594e25a2
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/594e25a2

Branch: refs/heads/spark
Commit: 594e25a230d8fabb7ae4e87faf3ceda689ef657e
Parents: 27bf8f0
Author: Gunther Hagleitner <gu...@apache.org>
Authored: Thu Sep 10 12:57:07 2015 -0700
Committer: Gunther Hagleitner <gu...@apache.org>
Committed: Thu Sep 10 12:57:07 2015 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/594e25a2/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
index f773cb9..19da1c3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
@@ -1015,6 +1015,9 @@ public class DagUtils {
     conf.set("mapred.partitioner.class", HiveConf.getVar(conf, HiveConf.ConfVars.HIVEPARTITIONER));
     conf.set("tez.runtime.partitioner.class", MRPartitioner.class.getName());
 
+    // Removing job credential entry/ cannot be set on the tasks
+    conf.unset("mapreduce.job.credentials.binary");
+
     Utilities.stripHivePasswordDetails(conf);
     return conf;
   }


[47/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_resolution.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_resolution.q.out b/ql/src/test/results/clientpositive/groupby_resolution.q.out
index 7df53f2..ea40014 100644
--- a/ql/src/test/results/clientpositive/groupby_resolution.q.out
+++ b/ql/src/test/results/clientpositive/groupby_resolution.q.out
@@ -15,12 +15,12 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: string)
+                key expressions: key (type: string)
                 sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+                Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
@@ -60,12 +60,12 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: string)
+                key expressions: key (type: string)
                 sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+                Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
@@ -106,10 +106,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: string)
+                key expressions: key (type: string)
                 sort order: +
                 Map-reduce partition columns: rand() (type: double)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -176,10 +176,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: string)
+                key expressions: key (type: string)
                 sort order: +
                 Map-reduce partition columns: rand() (type: double)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -245,11 +245,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -297,11 +297,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -350,11 +350,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -427,11 +427,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -607,22 +607,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (key < '12') (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: rand() (type: double)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_rollup1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_rollup1.q.out b/ql/src/test/results/clientpositive/groupby_rollup1.q.out
index 8e04d86..292e14d 100644
--- a/ql/src/test/results/clientpositive/groupby_rollup1.q.out
+++ b/ql/src/test/results/clientpositive/groupby_rollup1.q.out
@@ -116,11 +116,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), val (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, val
               Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: string), '0' (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT val)
+                keys: key (type: string), '0' (type: string), val (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 2 Data size: 60 Basic stats: COMPLETE Column stats: NONE
@@ -293,11 +293,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), val (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, val
               Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: string), '0' (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT val)
+                keys: key (type: string), '0' (type: string), val (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 2 Data size: 60 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_sort_10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_sort_10.q.out b/ql/src/test/results/clientpositive/groupby_sort_10.q.out
index 2fead26..db8899f 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_10.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_10.q.out
@@ -41,10 +41,10 @@ STAGE PLANS:
             Statistics: Num rows: 4 Data size: 30 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 4 Data size: 30 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: final
                 outputColumnNames: _col0
                 Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE
@@ -109,11 +109,11 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 60 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 8 Data size: 60 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 bucketGroup: true
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 8 Data size: 60 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_sort_11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_sort_11.q.out b/ql/src/test/results/clientpositive/groupby_sort_11.q.out
index adf585d..2d4a44e 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_11.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_11.q.out
@@ -41,12 +41,12 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col0)
+                aggregations: count(DISTINCT key)
                 bucketGroup: true
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
@@ -435,10 +435,10 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: string)
+                key expressions: key (type: string)
                 sort order: +
                 Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/groupby_sort_8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_sort_8.q.out b/ql/src/test/results/clientpositive/groupby_sort_8.q.out
index ec16eb0..5152385 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_8.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_8.q.out
@@ -56,12 +56,12 @@ STAGE PLANS:
             Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col0)
+                aggregations: count(DISTINCT key)
                 bucketGroup: true
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -122,11 +122,11 @@ STAGE PLANS:
             Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col0)
-                keys: _col0 (type: string)
+                aggregations: count(DISTINCT key)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/having.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/having.q.out b/ql/src/test/results/clientpositive/having.q.out
index 8682a46..35b2901 100644
--- a/ql/src/test/results/clientpositive/having.q.out
+++ b/ql/src/test/results/clientpositive/having.q.out
@@ -99,22 +99,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (UDFToDouble(key) <> 302.0) (type: boolean)
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                aggregations: max(value)
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: max(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: string)
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Group By Operator
           aggregations: max(VALUE._col0)
@@ -469,11 +465,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: max(_col1)
-                keys: _col0 (type: string)
+                aggregations: max(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -736,22 +732,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (UDFToDouble(key) > 300.0) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                aggregations: max(value)
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: max(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: string)
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Group By Operator
           aggregations: max(VALUE._col0)
@@ -930,11 +922,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: max(_col1)
-                keys: _col0 (type: string)
+                aggregations: max(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1192,11 +1184,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/having2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/having2.q.out b/ql/src/test/results/clientpositive/having2.q.out
index ba601f9..699d8ee 100644
--- a/ql/src/test/results/clientpositive/having2.q.out
+++ b/ql/src/test/results/clientpositive/having2.q.out
@@ -133,11 +133,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
               expressions: customer_name (type: string), customer_balance (type: double), order_quantity (type: double), discount (type: double)
-              outputColumnNames: _col0, _col1, _col2, _col3
+              outputColumnNames: customer_name, customer_balance, order_quantity, discount
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: sum(_col1), sum(_col2), count(_col3)
-                keys: _col0 (type: string)
+                aggregations: sum(customer_balance), sum(order_quantity), count(discount)
+                keys: customer_name (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -200,11 +200,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
               expressions: customer_name (type: string), customer_balance (type: double), order_quantity (type: double), discount (type: double)
-              outputColumnNames: _col0, _col1, _col2, _col3
+              outputColumnNames: customer_name, customer_balance, order_quantity, discount
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: sum(_col1), sum(_col2), count(_col3)
-                keys: _col0 (type: string)
+                aggregations: sum(customer_balance), sum(order_quantity), count(discount)
+                keys: customer_name (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/index_auto_mult_tables.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/index_auto_mult_tables.q.out b/ql/src/test/results/clientpositive/index_auto_mult_tables.q.out
index 7755ed5..2639edf 100644
--- a/ql/src/test/results/clientpositive/index_auto_mult_tables.q.out
+++ b/ql/src/test/results/clientpositive/index_auto_mult_tables.q.out
@@ -229,10 +229,10 @@ STAGE PLANS:
               predicate: (((((UDFToDouble(key) > 70.0) and (UDFToDouble(key) < 90.0)) and (UDFToDouble(key) > 80.0)) and (UDFToDouble(key) < 100.0)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
               Select Operator
                 expressions: _bucketname (type: string), _offset (type: bigint)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: _col1, _col2
                 Group By Operator
-                  aggregations: collect_set(_col1)
-                  keys: _col0 (type: string)
+                  aggregations: collect_set(_col2)
+                  keys: _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Reduce Output Operator
@@ -326,10 +326,10 @@ STAGE PLANS:
               predicate: (((((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) and (UDFToDouble(key) > 70.0)) and (UDFToDouble(key) < 90.0)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
               Select Operator
                 expressions: _bucketname (type: string), _offset (type: bigint)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: _col1, _col2
                 Group By Operator
-                  aggregations: collect_set(_col1)
-                  keys: _col0 (type: string)
+                  aggregations: collect_set(_col2)
+                  keys: _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/index_auto_self_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/index_auto_self_join.q.out b/ql/src/test/results/clientpositive/index_auto_self_join.q.out
index a214845..e8c23dc 100644
--- a/ql/src/test/results/clientpositive/index_auto_self_join.q.out
+++ b/ql/src/test/results/clientpositive/index_auto_self_join.q.out
@@ -137,10 +137,10 @@ STAGE PLANS:
               predicate: (((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
               Select Operator
                 expressions: _bucketname (type: string), _offset (type: bigint)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: _col1, _col2
                 Group By Operator
-                  aggregations: collect_set(_col1)
-                  keys: _col0 (type: string)
+                  aggregations: collect_set(_col2)
+                  keys: _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Reduce Output Operator
@@ -235,10 +235,10 @@ STAGE PLANS:
               predicate: (((UDFToDouble(key) > 70.0) and (UDFToDouble(key) < 90.0)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
               Select Operator
                 expressions: _bucketname (type: string), _offset (type: bigint)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: _col1, _col2
                 Group By Operator
-                  aggregations: collect_set(_col1)
-                  keys: _col0 (type: string)
+                  aggregations: collect_set(_col2)
+                  keys: _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/index_auto_update.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/index_auto_update.q.out b/ql/src/test/results/clientpositive/index_auto_update.q.out
index 70d0286..11af3f5 100644
--- a/ql/src/test/results/clientpositive/index_auto_update.q.out
+++ b/ql/src/test/results/clientpositive/index_auto_update.q.out
@@ -109,10 +109,10 @@ STAGE PLANS:
             alias: temp
             Select Operator
               expressions: key (type: string), INPUT__FILE__NAME (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint)
-              outputColumnNames: _col0, _col1, _col2
+              outputColumnNames: key, INPUT__FILE__NAME, BLOCK__OFFSET__INSIDE__FILE
               Group By Operator
-                aggregations: collect_set(_col2)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: collect_set(BLOCK__OFFSET__INSIDE__FILE)
+                keys: key (type: string), INPUT__FILE__NAME (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/index_bitmap_auto_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/index_bitmap_auto_partitioned.q.out b/ql/src/test/results/clientpositive/index_bitmap_auto_partitioned.q.out
index 65b6721..3df629e 100644
--- a/ql/src/test/results/clientpositive/index_bitmap_auto_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/index_bitmap_auto_partitioned.q.out
@@ -70,10 +70,10 @@ STAGE PLANS:
               predicate: ((UDFToDouble(key) = 86.0) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
               Select Operator
                 expressions: _bucketname (type: string), _offset (type: bigint)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: _col1, _col2
                 Group By Operator
-                  aggregations: collect_set(_col1)
-                  keys: _col0 (type: string)
+                  aggregations: collect_set(_col2)
+                  keys: _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/index_bitmap_compression.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/index_bitmap_compression.q.out b/ql/src/test/results/clientpositive/index_bitmap_compression.q.out
index 9673f08..37a2944 100644
--- a/ql/src/test/results/clientpositive/index_bitmap_compression.q.out
+++ b/ql/src/test/results/clientpositive/index_bitmap_compression.q.out
@@ -44,10 +44,10 @@ STAGE PLANS:
               predicate: (((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
               Select Operator
                 expressions: _bucketname (type: string), _offset (type: bigint)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: _col1, _col2
                 Group By Operator
-                  aggregations: collect_set(_col1)
-                  keys: _col0 (type: string)
+                  aggregations: collect_set(_col2)
+                  keys: _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
index c2f0810..a12dcd5 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
@@ -473,11 +473,11 @@ STAGE PLANS:
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
index 6aef463..1294766 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
@@ -81,11 +81,11 @@ STAGE PLANS:
             Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/join18.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join18.q.out b/ql/src/test/results/clientpositive/join18.q.out
index c3ab306..6a50604 100644
--- a/ql/src/test/results/clientpositive/join18.q.out
+++ b/ql/src/test/results/clientpositive/join18.q.out
@@ -43,11 +43,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -113,11 +113,11 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT value)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/join18_multi_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join18_multi_distinct.q.out b/ql/src/test/results/clientpositive/join18_multi_distinct.q.out
index a83102c..dfb0f68 100644
--- a/ql/src/test/results/clientpositive/join18_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/join18_multi_distinct.q.out
@@ -45,11 +45,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -115,11 +115,11 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1), count(DISTINCT _col0)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT value), count(DISTINCT key)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/join31.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join31.q.out b/ql/src/test/results/clientpositive/join31.q.out
index 21aa1de..70ca814 100644
--- a/ql/src/test/results/clientpositive/join31.q.out
+++ b/ql/src/test/results/clientpositive/join31.q.out
@@ -51,20 +51,16 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
+              Group By Operator
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)
@@ -264,20 +260,16 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
+              Group By Operator
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/limit_partition_metadataonly.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/limit_partition_metadataonly.q.out b/ql/src/test/results/clientpositive/limit_partition_metadataonly.q.out
index 904acfc..dbc5eac 100644
--- a/ql/src/test/results/clientpositive/limit_partition_metadataonly.q.out
+++ b/ql/src/test/results/clientpositive/limit_partition_metadataonly.q.out
@@ -550,10 +550,10 @@ STAGE PLANS:
             Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: hr (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: hr
               Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string)
+                keys: hr (type: string)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/limit_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/limit_pushdown.q.out b/ql/src/test/results/clientpositive/limit_pushdown.q.out
index d77bc60..40f6a48 100644
--- a/ql/src/test/results/clientpositive/limit_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/limit_pushdown.q.out
@@ -355,10 +355,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: cdouble (type: double)
-              outputColumnNames: _col0
+              outputColumnNames: cdouble
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: double)
+                keys: cdouble (type: double)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -438,11 +438,11 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctinyint (type: tinyint), cdouble (type: double)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: ctinyint, cdouble
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: tinyint), _col1 (type: double)
+                aggregations: count(DISTINCT cdouble)
+                keys: ctinyint (type: tinyint), cdouble (type: double)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -523,10 +523,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctinyint (type: tinyint), cdouble (type: double)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: ctinyint, cdouble
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: tinyint), _col1 (type: double)
+                keys: ctinyint (type: tinyint), cdouble (type: double)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -613,11 +613,11 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctinyint (type: tinyint), cstring1 (type: string), cstring2 (type: string)
-              outputColumnNames: _col0, _col1, _col2
+              outputColumnNames: ctinyint, cstring1, cstring2
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1), count(DISTINCT _col2)
-                keys: _col0 (type: tinyint), _col1 (type: string), _col2 (type: string)
+                aggregations: count(DISTINCT cstring1), count(DISTINCT cstring2)
+                keys: ctinyint (type: tinyint), cstring1 (type: string), cstring2 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -727,11 +727,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: value (type: string), key (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: value, key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col1)
-                keys: _col0 (type: string)
+                aggregations: sum(key)
+                keys: value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1012,16 +1012,16 @@ STAGE PLANS:
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: value (type: string), key (type: string)
-              outputColumnNames: _col0, _col1
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: string)
+                key expressions: value (type: string)
                 sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+                Map-reduce partition columns: value (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 TopN Hash Memory Usage: 0.3
-                value expressions: _col1 (type: string)
+                value expressions: key (type: string)
       Reduce Operator Tree:
         Group By Operator
           aggregations: sum(VALUE._col0)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/lineage2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lineage2.q.out b/ql/src/test/results/clientpositive/lineage2.q.out
index 549b5f6..4184a83 100644
--- a/ql/src/test/results/clientpositive/lineage2.q.out
+++ b/ql/src/test/results/clientpositive/lineage2.q.out
@@ -634,7 +634,7 @@ having count(a.c2) > 0
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_l2
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","hash":"0ae7aa4a0cbd1283210fa79e8a19104a","queryText":"select sum(a.c1) over (partition by a.c1 order by a.id)\nfrom dest_l2 a\nwhere a.c2 != 10\ngroup by a.c1, a.c2, a.id\nhaving count(a.c2) > 0","edges":[{"sources":[1,2,3],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) $f0) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) $f0)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) $f2)))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0],"expression":"(a.c2 <> 10)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0],"expression":"(count(default.dest_l2.c2) > 0)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c0"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l2.c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default
 .dest_l2.id"}]}
+{"version":"1.0","engine":"mr","hash":"0ae7aa4a0cbd1283210fa79e8a19104a","queryText":"select sum(a.c1) over (partition by a.c1 order by a.id)\nfrom dest_l2 a\nwhere a.c2 != 10\ngroup by a.c1, a.c2, a.id\nhaving count(a.c2) > 0","edges":[{"sources":[1,2,3],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) c1) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) c1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) id)))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0],"expression":"(a.c2 <> 10)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0],"expression":"(count(default.dest_l2.c2) > 0)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c0"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l2.c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.de
 st_l2.id"}]}
 1
 PREHOOK: query: select sum(a.c1), count(b.c1), b.c2, b.c3
 from dest_l2 a join dest_l3 b on (a.id = b.id)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/lineage3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lineage3.q.out b/ql/src/test/results/clientpositive/lineage3.q.out
index 6fd2aa4..708abee 100644
--- a/ql/src/test/results/clientpositive/lineage3.q.out
+++ b/ql/src/test/results/clientpositive/lineage3.q.out
@@ -180,7 +180,7 @@ PREHOOK: Input: default@src1
 #### A masked pattern was here ####
 {"version":"1.0","engine":"mr","hash":"8bf193b0658183be94e2428a79d91d10","queryText":"select * from src1 a\nwhere exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > UDFToDouble(300))","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 311	val_311
-Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: select key, value from src1
 where key not in (select key+18 from src1) order by key
 PREHOOK: type: QUERY
@@ -317,6 +317,6 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@dest_v3
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","hash":"40bccc0722002f798d0548b59e369e83","queryText":"select * from dest_v3 limit 2","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) $f0) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) $f1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) $f1)))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[8],"targets":[0,1,2],"expression":"(a.cboolean2 = true)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(a.cint = a.cint)","edgeType":"PREDICATE"},{"sources":[9],"targets":[0,1,2],"expression":"(a.cfloat > 0.0)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edge
 Type":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"}]}
+{"version":"1.0","engine":"mr","hash":"40bccc0722002f798d0548b59e369e83","queryText":"select * from dest_v3 limit 2","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) ctinyint) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) csmallint)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) csmallint)))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[8],"targets":[0,1,2],"expression":"(a.cboolean2 = true)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(a.cint = a.cint)","edgeType":"PREDICATE"},{"sources":[9],"targets":[0,1,2],"expression":"(a.cfloat > 0.0)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.c
 int) > 10)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"}]}
 38	216	false
 38	229	true

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out b/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
index 753729f..8f1d94e 100644
--- a/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
@@ -451,7 +451,7 @@ STAGE PLANS:
               name: default.fact_daily
             name: default.fact_daily
       Truncated Path -> Alias:
-        /fact_daily/ds=1/hr=1 [$hdt$_0:fact_daily]
+        /fact_daily/ds=1/hr=1 [fact_daily]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out b/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
index 9775f30..e92f307 100644
--- a/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
@@ -566,11 +566,11 @@ STAGE PLANS:
   Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:src 
+        $hdt$_1:src 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:src 
+        $hdt$_1:src 
           TableScan
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -619,11 +619,11 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_2:src 
+        $hdt$_2:src 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_2:src 
+        $hdt$_2:src 
           TableScan
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -651,22 +651,18 @@ STAGE PLANS:
                 1 _col0 (type: string)
               outputColumnNames: _col2
               Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: _col2 (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: _col2 (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Local Work:
         Map Reduce Local Work
       Reduce Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/metadata_only_queries.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/metadata_only_queries.q.out b/ql/src/test/results/clientpositive/metadata_only_queries.q.out
index 5907f4a..2dcd437 100644
--- a/ql/src/test/results/clientpositive/metadata_only_queries.q.out
+++ b/ql/src/test/results/clientpositive/metadata_only_queries.q.out
@@ -431,10 +431,10 @@ STAGE PLANS:
             Statistics: Num rows: 9489 Data size: 978785 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ts (type: timestamp)
-              outputColumnNames: _col0
+              outputColumnNames: ts
               Statistics: Num rows: 9489 Data size: 978785 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(ts)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/metadataonly1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/metadataonly1.q.out b/ql/src/test/results/clientpositive/metadataonly1.q.out
index 0d758a5..727b2e7 100644
--- a/ql/src/test/results/clientpositive/metadataonly1.q.out
+++ b/ql/src/test/results/clientpositive/metadataonly1.q.out
@@ -43,10 +43,10 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: ds (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: ds
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: max(_col0)
+                aggregations: max(ds)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -144,10 +144,10 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: ds (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: ds
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: max(_col0)
+                aggregations: max(ds)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -158,7 +158,7 @@ STAGE PLANS:
                   value expressions: _col0 (type: string)
                   auto parallelism: false
       Path -> Alias:
-        -mr-10003default.test1{ds=1} [$hdt$_0:test1]
+        -mr-10003default.test1{ds=1} [test1]
       Path -> Partition:
         -mr-10003default.test1{ds=1} 
           Partition
@@ -200,7 +200,7 @@ STAGE PLANS:
               name: default.test1
             name: default.test1
       Truncated Path -> Alias:
-        -mr-10003default.test1{ds=1} [$hdt$_0:test1]
+        -mr-10003default.test1{ds=1} [test1]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -284,11 +284,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: ds (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: ds
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col0)
-                keys: _col0 (type: string)
+                aggregations: count(DISTINCT ds)
+                keys: ds (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -299,7 +299,7 @@ STAGE PLANS:
                   tag: -1
                   auto parallelism: false
       Path -> Alias:
-        -mr-10003default.test1{ds=1} [$hdt$_0:test1]
+        -mr-10003default.test1{ds=1} [test1]
       Path -> Partition:
         -mr-10003default.test1{ds=1} 
           Partition
@@ -341,7 +341,7 @@ STAGE PLANS:
               name: default.test1
             name: default.test1
       Truncated Path -> Alias:
-        -mr-10003default.test1{ds=1} [$hdt$_0:test1]
+        -mr-10003default.test1{ds=1} [test1]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -425,10 +425,10 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: ds (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: ds
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(ds)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -482,7 +482,7 @@ STAGE PLANS:
               name: default.test1
             name: default.test1
       Truncated Path -> Alias:
-        /test1/ds=1 [$hdt$_0:test1]
+        /test1/ds=1 [test1]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -604,10 +604,10 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: ds (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: ds
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: max(_col0)
+                aggregations: max(ds)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -618,8 +618,8 @@ STAGE PLANS:
                   value expressions: _col0 (type: string)
                   auto parallelism: false
       Path -> Alias:
-        -mr-10005default.test1{ds=1} [$hdt$_0:$hdt$_1:$hdt$_1:a2]
-        -mr-10006default.test1{ds=2} [$hdt$_0:$hdt$_1:$hdt$_1:a2]
+        -mr-10005default.test1{ds=1} [$hdt$_1:a2]
+        -mr-10006default.test1{ds=2} [$hdt$_1:a2]
       Path -> Partition:
         -mr-10005default.test1{ds=1} 
           Partition
@@ -700,8 +700,8 @@ STAGE PLANS:
               name: default.test1
             name: default.test1
       Truncated Path -> Alias:
-        -mr-10005default.test1{ds=1} [$hdt$_0:$hdt$_1:$hdt$_1:a2]
-        -mr-10006default.test1{ds=2} [$hdt$_0:$hdt$_1:$hdt$_1:a2]
+        -mr-10005default.test1{ds=1} [$hdt$_1:a2]
+        -mr-10006default.test1{ds=2} [$hdt$_1:a2]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -862,8 +862,8 @@ STAGE PLANS:
               name: default.test1
             name: default.test1
       Truncated Path -> Alias:
-        /test1/ds=1 [$hdt$_0:$hdt$_0:a2]
-        /test1/ds=2 [$hdt$_0:$hdt$_0:a2]
+        /test1/ds=1 [$hdt$_0:a2]
+        /test1/ds=2 [$hdt$_0:a2]
 #### A masked pattern was here ####
       Needs Tagging: true
       Reduce Operator Tree:
@@ -1053,11 +1053,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: ds (type: string), hr (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: ds, hr
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT hr)
+                keys: ds (type: string), hr (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1069,9 +1069,9 @@ STAGE PLANS:
                   tag: -1
                   auto parallelism: false
       Path -> Alias:
-        -mr-10003default.test2{ds=1, hr=1} [$hdt$_0:test2]
-        -mr-10004default.test2{ds=1, hr=2} [$hdt$_0:test2]
-        -mr-10005default.test2{ds=1, hr=3} [$hdt$_0:test2]
+        -mr-10003default.test2{ds=1, hr=1} [test2]
+        -mr-10004default.test2{ds=1, hr=2} [test2]
+        -mr-10005default.test2{ds=1, hr=3} [test2]
       Path -> Partition:
         -mr-10003default.test2{ds=1, hr=1} 
           Partition
@@ -1194,9 +1194,9 @@ STAGE PLANS:
               name: default.test2
             name: default.test2
       Truncated Path -> Alias:
-        -mr-10003default.test2{ds=1, hr=1} [$hdt$_0:test2]
-        -mr-10004default.test2{ds=1, hr=2} [$hdt$_0:test2]
-        -mr-10005default.test2{ds=1, hr=3} [$hdt$_0:test2]
+        -mr-10003default.test2{ds=1, hr=1} [test2]
+        -mr-10004default.test2{ds=1, hr=2} [test2]
+        -mr-10005default.test2{ds=1, hr=3} [test2]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -1291,11 +1291,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: ds (type: string), hr (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: ds, hr
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(hr)
+                keys: ds (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1434,9 +1434,9 @@ STAGE PLANS:
               name: default.test2
             name: default.test2
       Truncated Path -> Alias:
-        /test2/ds=1/hr=1 [$hdt$_0:test2]
-        /test2/ds=1/hr=2 [$hdt$_0:test2]
-        /test2/ds=1/hr=3 [$hdt$_0:test2]
+        /test2/ds=1/hr=1 [test2]
+        /test2/ds=1/hr=2 [test2]
+        /test2/ds=1/hr=3 [test2]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -1524,10 +1524,10 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: ds (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: ds
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: max(_col0)
+                aggregations: max(ds)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -1538,8 +1538,8 @@ STAGE PLANS:
                   value expressions: _col0 (type: string)
                   auto parallelism: false
       Path -> Alias:
-        -mr-10003default.test1{ds=1} [$hdt$_0:test1]
-        -mr-10004default.test1{ds=2} [$hdt$_0:test1]
+        -mr-10003default.test1{ds=1} [test1]
+        -mr-10004default.test1{ds=2} [test1]
       Path -> Partition:
         -mr-10003default.test1{ds=1} 
           Partition
@@ -1620,8 +1620,8 @@ STAGE PLANS:
               name: default.test1
             name: default.test1
       Truncated Path -> Alias:
-        -mr-10003default.test1{ds=1} [$hdt$_0:test1]
-        -mr-10004default.test1{ds=2} [$hdt$_0:test1]
+        -mr-10003default.test1{ds=1} [test1]
+        -mr-10004default.test1{ds=2} [test1]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -1764,11 +1764,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: ds (type: string), hr (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: ds, hr
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT hr)
+                keys: ds (type: string), hr (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1780,11 +1780,11 @@ STAGE PLANS:
                   tag: -1
                   auto parallelism: false
       Path -> Alias:
-        -mr-10003default.test2{ds=01_10_10, hr=01} [$hdt$_0:test2]
-        -mr-10004default.test2{ds=01_10_20, hr=02} [$hdt$_0:test2]
-        -mr-10005default.test2{ds=1, hr=1} [$hdt$_0:test2]
-        -mr-10006default.test2{ds=1, hr=2} [$hdt$_0:test2]
-        -mr-10007default.test2{ds=1, hr=3} [$hdt$_0:test2]
+        -mr-10003default.test2{ds=01_10_10, hr=01} [test2]
+        -mr-10004default.test2{ds=01_10_20, hr=02} [test2]
+        -mr-10005default.test2{ds=1, hr=1} [test2]
+        -mr-10006default.test2{ds=1, hr=2} [test2]
+        -mr-10007default.test2{ds=1, hr=3} [test2]
       Path -> Partition:
         -mr-10003default.test2{ds=01_10_10, hr=01} 
           Partition
@@ -1987,11 +1987,11 @@ STAGE PLANS:
               name: default.test2
             name: default.test2
       Truncated Path -> Alias:
-        -mr-10003default.test2{ds=01_10_10, hr=01} [$hdt$_0:test2]
-        -mr-10004default.test2{ds=01_10_20, hr=02} [$hdt$_0:test2]
-        -mr-10005default.test2{ds=1, hr=1} [$hdt$_0:test2]
-        -mr-10006default.test2{ds=1, hr=2} [$hdt$_0:test2]
-        -mr-10007default.test2{ds=1, hr=3} [$hdt$_0:test2]
+        -mr-10003default.test2{ds=01_10_10, hr=01} [test2]
+        -mr-10004default.test2{ds=01_10_20, hr=02} [test2]
+        -mr-10005default.test2{ds=1, hr=1} [test2]
+        -mr-10006default.test2{ds=1, hr=2} [test2]
+        -mr-10007default.test2{ds=1, hr=3} [test2]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator


[39/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/having.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/having.q.out b/ql/src/test/results/clientpositive/tez/having.q.out
index 80f02de..31de85a 100644
--- a/ql/src/test/results/clientpositive/tez/having.q.out
+++ b/ql/src/test/results/clientpositive/tez/having.q.out
@@ -110,22 +110,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (UDFToDouble(key) <> 302.0) (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      aggregations: max(value)
+                      keys: key (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: max(_col1)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: string)
+                        value expressions: _col1 (type: string)
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
@@ -486,11 +482,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: max(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -759,22 +755,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (UDFToDouble(key) > 300.0) (type: boolean)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      aggregations: max(value)
+                      keys: key (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: max(_col1)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: string)
+                        value expressions: _col1 (type: string)
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
@@ -959,11 +951,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: max(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1227,11 +1219,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: count(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/limit_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/limit_pushdown.q.out b/ql/src/test/results/clientpositive/tez/limit_pushdown.q.out
index 01ccae7..14d57e3 100644
--- a/ql/src/test/results/clientpositive/tez/limit_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/tez/limit_pushdown.q.out
@@ -384,10 +384,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cdouble (type: double)
-                    outputColumnNames: _col0
+                    outputColumnNames: cdouble
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: double)
+                      keys: cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -473,10 +473,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint), cdouble (type: double)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: ctinyint, cdouble
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: tinyint), _col1 (type: double)
+                      keys: ctinyint (type: tinyint), cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -567,10 +567,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint), cdouble (type: double)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: ctinyint, cdouble
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: tinyint), _col1 (type: double)
+                      keys: ctinyint (type: tinyint), cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -663,11 +663,11 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint), cstring1 (type: string), cstring2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
+                    outputColumnNames: ctinyint, cstring1, cstring2
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1), count(DISTINCT _col2)
-                      keys: _col0 (type: tinyint), _col1 (type: string), _col2 (type: string)
+                      aggregations: count(DISTINCT cstring1), count(DISTINCT cstring2)
+                      keys: ctinyint (type: tinyint), cstring1 (type: string), cstring2 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -783,11 +783,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: value (type: string), key (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: value, key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: sum(key)
+                      keys: value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1042,16 +1042,16 @@ STAGE PLANS:
                   alias: src
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: value (type: string), key (type: string)
-                    outputColumnNames: _col0, _col1
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: string)
+                      key expressions: value (type: string)
                       sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                      Map-reduce partition columns: value (type: string)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       TopN Hash Memory Usage: 0.3
-                      value expressions: _col1 (type: string)
+                      value expressions: key (type: string)
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out b/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
index 23df010..0bf689b 100644
--- a/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
@@ -617,22 +617,18 @@ STAGE PLANS:
                             1 Map 4
                           Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                           HybridGraceHashJoin: true
-                          Select Operator
-                            expressions: _col2 (type: string)
-                            outputColumnNames: _col0
+                          Group By Operator
+                            aggregations: count()
+                            keys: _col2 (type: string)
+                            mode: hash
+                            outputColumnNames: _col0, _col1
                             Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                            Group By Operator
-                              aggregations: count()
-                              keys: _col0 (type: string)
-                              mode: hash
-                              outputColumnNames: _col0, _col1
+                            Reduce Output Operator
+                              key expressions: _col0 (type: string)
+                              sort order: +
+                              Map-reduce partition columns: _col0 (type: string)
                               Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                              Reduce Output Operator
-                                key expressions: _col0 (type: string)
-                                sort order: +
-                                Map-reduce partition columns: _col0 (type: string)
-                                Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                                value expressions: _col1 (type: bigint)
+                              value expressions: _col1 (type: bigint)
         Map 3 
             Map Operator Tree:
                 TableScan

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/metadata_only_queries.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/metadata_only_queries.q.out b/ql/src/test/results/clientpositive/tez/metadata_only_queries.q.out
index aaea52e..f43440e 100644
--- a/ql/src/test/results/clientpositive/tez/metadata_only_queries.q.out
+++ b/ql/src/test/results/clientpositive/tez/metadata_only_queries.q.out
@@ -448,10 +448,10 @@ STAGE PLANS:
                   Statistics: Num rows: 9489 Data size: 978785 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ts (type: timestamp)
-                    outputColumnNames: _col0
+                    outputColumnNames: ts
                     Statistics: Num rows: 9489 Data size: 978785 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(ts)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/metadataonly1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/metadataonly1.q.out b/ql/src/test/results/clientpositive/tez/metadataonly1.q.out
index 927b686..b55cb7e 100644
--- a/ql/src/test/results/clientpositive/tez/metadataonly1.q.out
+++ b/ql/src/test/results/clientpositive/tez/metadataonly1.q.out
@@ -48,10 +48,10 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -155,10 +155,10 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -301,11 +301,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col0)
-                      keys: _col0 (type: string)
+                      aggregations: count(DISTINCT ds)
+                      keys: ds (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -448,10 +448,10 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -736,10 +736,10 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -1017,11 +1017,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: ds (type: string), hr (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: ds, hr
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1)
-                      keys: _col0 (type: string), _col1 (type: string)
+                      aggregations: count(DISTINCT hr)
+                      keys: ds (type: string), hr (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1261,11 +1261,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: ds (type: string), hr (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: ds, hr
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: count(hr)
+                      keys: ds (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1500,10 +1500,10 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -1746,11 +1746,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: ds (type: string), hr (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: ds, hr
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1)
-                      keys: _col0 (type: string), _col1 (type: string)
+                      aggregations: count(DISTINCT hr)
+                      keys: ds (type: string), hr (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/mrr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/mrr.q.out b/ql/src/test/results/clientpositive/tez/mrr.q.out
index efbd02d..faace21 100644
--- a/ql/src/test/results/clientpositive/tez/mrr.q.out
+++ b/ql/src/test/results/clientpositive/tez/mrr.q.out
@@ -27,11 +27,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: count(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1304,22 +1304,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      aggregations: count(value)
+                      keys: key (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: count(_col1)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: bigint)
+                        value expressions: _col1 (type: bigint)
         Map 6 
             Map Operator Tree:
                 TableScan
@@ -1328,22 +1324,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      aggregations: count(value)
+                      keys: key (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: count(_col1)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: bigint)
+                        value expressions: _col1 (type: bigint)
         Map 9 
             Map Operator Tree:
                 TableScan
@@ -1352,22 +1344,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      aggregations: count(value)
+                      keys: key (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: count(_col1)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: bigint)
+                        value expressions: _col1 (type: bigint)
         Reducer 10 
             Reduce Operator Tree:
               Group By Operator
@@ -1692,22 +1680,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      aggregations: count(value)
+                      keys: key (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: count(_col1)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: bigint)
+                        value expressions: _col1 (type: bigint)
         Map 3 
             Map Operator Tree:
                 TableScan

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/selectDistinctStar.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/selectDistinctStar.q.out b/ql/src/test/results/clientpositive/tez/selectDistinctStar.q.out
index 0a091da..22ac115 100644
--- a/ql/src/test/results/clientpositive/tez/selectDistinctStar.q.out
+++ b/ql/src/test/results/clientpositive/tez/selectDistinctStar.q.out
@@ -24,10 +24,10 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string), _col1 (type: string)
+                      keys: key (type: string), value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -718,20 +718,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (key < '3') (type: boolean)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      keys: key (type: string), value (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: string), _col1 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: string)
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                          Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
@@ -2474,10 +2470,10 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string), _col1 (type: string)
+                      keys: key (type: string), value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -3164,20 +3160,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (key < '3') (type: boolean)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      keys: key (type: string), value (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: string), _col1 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: string)
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                          Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/stats_only_null.q.out b/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
index eabda89..0412052 100644
--- a/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
+++ b/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
@@ -90,10 +90,10 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: a (type: double), b (type: int), c (type: string), d (type: smallint)
-                    outputColumnNames: _col0, _col1, _col2, _col3
+                    outputColumnNames: a, b, c, d
                     Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(), count(_col0), count(_col1), count(_col2), count(_col3)
+                      aggregations: count(), count(a), count(b), count(c), count(d)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
@@ -146,10 +146,10 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: a (type: double), b (type: int), c (type: string), d (type: smallint)
-                    outputColumnNames: _col0, _col1, _col2, _col3
+                    outputColumnNames: a, b, c, d
                     Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(), count(_col0), count(_col1), count(_col2), count(_col3)
+                      aggregations: count(), count(a), count(b), count(c), count(d)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/subquery_in.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/subquery_in.q.out b/ql/src/test/results/clientpositive/tez/subquery_in.q.out
index 2b1237b..add2c52 100644
--- a/ql/src/test/results/clientpositive/tez/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/tez/subquery_in.q.out
@@ -646,20 +646,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: ((key > '9') and value is not null) (type: boolean)
                     Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      keys: key (type: string), value (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: string), _col1 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: string)
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                          Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Merge Join Operator
@@ -829,20 +825,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: l_partkey is not null (type: boolean)
                     Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: l_partkey (type: int)
+                    Group By Operator
+                      keys: l_partkey (type: int)
+                      mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: int)
-                        mode: hash
-                        outputColumnNames: _col0
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Merge Join Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/tez_dml.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/tez_dml.q.out b/ql/src/test/results/clientpositive/tez/tez_dml.q.out
index ae85292..728eb6b 100644
--- a/ql/src/test/results/clientpositive/tez/tez_dml.q.out
+++ b/ql/src/test/results/clientpositive/tez/tez_dml.q.out
@@ -30,11 +30,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: value (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
-                      keys: _col0 (type: string)
+                      aggregations: count(value)
+                      keys: value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/union5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/union5.q.out b/ql/src/test/results/clientpositive/tez/union5.q.out
index 327195c..9939b7d 100644
--- a/ql/src/test/results/clientpositive/tez/union5.q.out
+++ b/ql/src/test/results/clientpositive/tez/union5.q.out
@@ -31,33 +31,37 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: key (type: string)
+                    outputColumnNames: key
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(1)
+                      aggregations: count(key)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         sort order: 
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
         Map 5 
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: key (type: string)
+                    outputColumnNames: key
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(1)
+                      aggregations: count(key)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         sort order: 
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
@@ -65,22 +69,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst1' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
         Reducer 4 
             Reduce Operator Tree:
@@ -89,10 +93,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -103,22 +107,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst2' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
         Union 3 
             Vertex: Union 3

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/union7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/union7.q.out b/ql/src/test/results/clientpositive/tez/union7.q.out
index c5ece62..7a02993 100644
--- a/ql/src/test/results/clientpositive/tez/union7.q.out
+++ b/ql/src/test/results/clientpositive/tez/union7.q.out
@@ -31,17 +31,19 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: key (type: string)
+                    outputColumnNames: key
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(1)
+                      aggregations: count(key)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         sort order: 
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
         Map 5 
             Map Operator Tree:
@@ -57,12 +59,12 @@ STAGE PLANS:
                       keys: _col0 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
@@ -70,22 +72,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst1' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
         Reducer 4 
             Reduce Operator Tree:
@@ -94,10 +96,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/unionDistinct_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/unionDistinct_1.q.out b/ql/src/test/results/clientpositive/tez/unionDistinct_1.q.out
index cab3a28..bcae2fc 100644
--- a/ql/src/test/results/clientpositive/tez/unionDistinct_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/unionDistinct_1.q.out
@@ -14059,11 +14059,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: string)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -14202,11 +14202,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: string)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_aggregate_9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_aggregate_9.q.out b/ql/src/test/results/clientpositive/tez/vector_aggregate_9.q.out
index 382380e..d6a8517 100644
--- a/ql/src/test/results/clientpositive/tez/vector_aggregate_9.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_aggregate_9.q.out
@@ -125,10 +125,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: dc (type: decimal(38,18))
-                    outputColumnNames: _col0
+                    outputColumnNames: dc
                     Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0), max(_col0), sum(_col0), avg(_col0)
+                      aggregations: min(dc), max(dc), sum(dc), avg(dc)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_binary_join_groupby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_binary_join_groupby.q.out b/ql/src/test/results/clientpositive/tez/vector_binary_join_groupby.q.out
index 8dcd40d..ebc0466 100644
--- a/ql/src/test/results/clientpositive/tez/vector_binary_join_groupby.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_binary_join_groupby.q.out
@@ -221,11 +221,11 @@ STAGE PLANS:
                   Statistics: Num rows: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: bin (type: binary)
-                    outputColumnNames: _col0
+                    outputColumnNames: bin
                     Statistics: Num rows: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: binary)
+                      keys: bin (type: binary)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_count_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_count_distinct.q.out b/ql/src/test/results/clientpositive/tez/vector_count_distinct.q.out
index 95863a3..f46bcd9 100644
--- a/ql/src/test/results/clientpositive/tez/vector_count_distinct.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_count_distinct.q.out
@@ -1258,10 +1258,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ws_order_number (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: ws_order_number
                     Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int)
+                      keys: ws_order_number (type: int)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_decimal_aggregate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_decimal_aggregate.q.out b/ql/src/test/results/clientpositive/tez/vector_decimal_aggregate.q.out
index 2e9c232..13e047b 100644
--- a/ql/src/test/results/clientpositive/tez/vector_decimal_aggregate.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_decimal_aggregate.q.out
@@ -54,11 +54,11 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cint (type: int), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14))
-                    outputColumnNames: _col0, _col1, _col2
+                    outputColumnNames: cint, cdecimal1, cdecimal2
                     Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1), max(_col1), min(_col1), sum(_col1), count(_col2), max(_col2), min(_col2), sum(_col2), count()
-                      keys: _col0 (type: int)
+                      aggregations: count(cdecimal1), max(cdecimal1), min(cdecimal1), sum(cdecimal1), count(cdecimal2), max(cdecimal2), min(cdecimal2), sum(cdecimal2), count()
+                      keys: cint (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                       Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
@@ -159,11 +159,11 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cint (type: int), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14))
-                    outputColumnNames: _col0, _col1, _col2
+                    outputColumnNames: cint, cdecimal1, cdecimal2
                     Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1), max(_col1), min(_col1), sum(_col1), avg(_col1), stddev_pop(_col1), stddev_samp(_col1), count(_col2), max(_col2), min(_col2), sum(_col2), avg(_col2), stddev_pop(_col2), stddev_samp(_col2), count()
-                      keys: _col0 (type: int)
+                      aggregations: count(cdecimal1), max(cdecimal1), min(cdecimal1), sum(cdecimal1), avg(cdecimal1), stddev_pop(cdecimal1), stddev_samp(cdecimal1), count(cdecimal2), max(cdecimal2), min(cdecimal2), sum(cdecimal2), avg(cdecimal2), stddev_pop(cdecimal2), stddev_samp(cdecimal2), count()
+                      keys: cint (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15
                       Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_decimal_precision.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_decimal_precision.q.out b/ql/src/test/results/clientpositive/tez/vector_decimal_precision.q.out
index 4e1b654..8b6614e 100644
--- a/ql/src/test/results/clientpositive/tez/vector_decimal_precision.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_decimal_precision.q.out
@@ -567,10 +567,10 @@ STAGE PLANS:
                   Statistics: Num rows: 75 Data size: 3472 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: dec (type: decimal(20,10))
-                    outputColumnNames: _col0
+                    outputColumnNames: dec
                     Statistics: Num rows: 75 Data size: 3472 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: avg(_col0), sum(_col0)
+                      aggregations: avg(dec), sum(dec)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_decimal_udf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_decimal_udf.q.out b/ql/src/test/results/clientpositive/tez/vector_decimal_udf.q.out
index 13f0de2..dc929d3 100644
--- a/ql/src/test/results/clientpositive/tez/vector_decimal_udf.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_decimal_udf.q.out
@@ -1639,11 +1639,11 @@ STAGE PLANS:
                   Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: value (type: int), key (type: decimal(20,10))
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: value, key
                     Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col1), count(_col1), avg(_col1)
-                      keys: _col0 (type: int)
+                      aggregations: sum(key), count(key), avg(key)
+                      keys: value (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
@@ -2317,11 +2317,11 @@ STAGE PLANS:
                   Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: value (type: int), key (type: decimal(20,10))
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: value, key
                     Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: stddev(_col1), variance(_col1)
-                      keys: _col0 (type: int)
+                      aggregations: stddev(key), variance(key)
+                      keys: value (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
@@ -2403,11 +2403,11 @@ STAGE PLANS:
                   Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: value (type: int), key (type: decimal(20,10))
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: value, key
                     Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: stddev_samp(_col1), var_samp(_col1)
-                      keys: _col0 (type: int)
+                      aggregations: stddev_samp(key), var_samp(key)
+                      keys: value (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
@@ -2554,10 +2554,10 @@ STAGE PLANS:
                   Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: decimal(20,10))
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0)
+                      aggregations: min(key)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
@@ -2621,10 +2621,10 @@ STAGE PLANS:
                   Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: decimal(20,10))
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(key)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
@@ -2688,10 +2688,10 @@ STAGE PLANS:
                   Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: decimal(20,10))
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(key)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_distinct_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_distinct_2.q.out b/ql/src/test/results/clientpositive/tez/vector_distinct_2.q.out
index 1671ddf..6c31294 100644
--- a/ql/src/test/results/clientpositive/tez/vector_distinct_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_distinct_2.q.out
@@ -128,34 +128,38 @@ STAGE PLANS:
                   alias: vectortab2korc
                   Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: s (type: string), t (type: tinyint)
-                    outputColumnNames: _col0, _col1
+                    expressions: t (type: tinyint), s (type: string)
+                    outputColumnNames: t, s
                     Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string), _col1 (type: tinyint)
+                      keys: t (type: tinyint), s (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: string), _col1 (type: tinyint)
+                        key expressions: _col0 (type: tinyint), _col1 (type: string)
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
+                        Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: string)
                         Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
+                Select Operator
+                  expressions: _col1 (type: string), _col0 (type: tinyint)
+                  outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
             Execution mode: vectorized
 
   Stage: Stage-0

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_groupby_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_groupby_3.q.out b/ql/src/test/results/clientpositive/tez/vector_groupby_3.q.out
index 9b9dcdd..6aa39f1 100644
--- a/ql/src/test/results/clientpositive/tez/vector_groupby_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_groupby_3.q.out
@@ -128,19 +128,19 @@ STAGE PLANS:
                   alias: vectortab2korc
                   Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: s (type: string), t (type: tinyint), b (type: bigint)
-                    outputColumnNames: _col0, _col1, _col2
+                    expressions: t (type: tinyint), s (type: string), b (type: bigint)
+                    outputColumnNames: t, s, b
                     Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col2)
-                      keys: _col0 (type: string), _col1 (type: tinyint)
+                      aggregations: max(b)
+                      keys: t (type: tinyint), s (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: string), _col1 (type: tinyint)
+                        key expressions: _col0 (type: tinyint), _col1 (type: string)
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
+                        Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: string)
                         Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col2 (type: bigint)
             Execution mode: vectorized
@@ -148,17 +148,21 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: max(VALUE._col0)
-                keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
+                Select Operator
+                  expressions: _col1 (type: string), _col0 (type: tinyint), _col2 (type: bigint)
+                  outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
             Execution mode: vectorized
 
   Stage: Stage-0

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_groupby_reduce.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_groupby_reduce.q.out b/ql/src/test/results/clientpositive/tez/vector_groupby_reduce.q.out
index ba2fb77..814ee39 100644
--- a/ql/src/test/results/clientpositive/tez/vector_groupby_reduce.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_groupby_reduce.q.out
@@ -245,10 +245,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ss_ticket_number (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: ss_ticket_number
                     Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int)
+                      keys: ss_ticket_number (type: int)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE
@@ -369,10 +369,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ss_ticket_number (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: ss_ticket_number
                     Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int)
+                      keys: ss_ticket_number (type: int)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_grouping_sets.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_grouping_sets.q.out b/ql/src/test/results/clientpositive/tez/vector_grouping_sets.q.out
index 688d2ac..6d39d02 100644
--- a/ql/src/test/results/clientpositive/tez/vector_grouping_sets.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_grouping_sets.q.out
@@ -126,10 +126,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: s_store_id (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: s_store_id
                     Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string), '0' (type: string)
+                      keys: s_store_id (type: string), '0' (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE
@@ -207,10 +207,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: s_store_id (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: s_store_id
                     Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string), '0' (type: string)
+                      keys: s_store_id (type: string), '0' (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_mapjoin_reduce.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_mapjoin_reduce.q.out b/ql/src/test/results/clientpositive/tez/vector_mapjoin_reduce.q.out
index e7fec82..d33b0ed 100644
--- a/ql/src/test/results/clientpositive/tez/vector_mapjoin_reduce.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_mapjoin_reduce.q.out
@@ -109,20 +109,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: l_partkey is not null (type: boolean)
                     Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: l_partkey (type: int)
+                    Group By Operator
+                      keys: l_partkey (type: int)
+                      mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: int)
-                        mode: hash
-                        outputColumnNames: _col0
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
         Reducer 4 
             Reduce Operator Tree:
               Group By Operator
@@ -252,20 +248,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: l_partkey is not null (type: boolean)
                     Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: l_partkey (type: int)
+                    Group By Operator
+                      keys: l_partkey (type: int)
+                      mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: int)
-                        mode: hash
-                        outputColumnNames: _col0
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
         Reducer 4 
             Reduce Operator Tree:
               Group By Operator


[03/50] [abbrv] hive git commit: HIVE-11482 : Adds retrying thrift client for HiveServer2 (Akshay Goyal, reviewed by Amareshwari)

Posted by xu...@apache.org.
HIVE-11482 : Adds retrying thrift client for HiveServer2 (Akshay Goyal, reviewed by Amareshwari)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9b11caff
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9b11caff
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9b11caff

Branch: refs/heads/spark
Commit: 9b11caff8b61697c88caa1ed5606c665624f3290
Parents: d94c0f6
Author: Akshay Goyal <ak...@gmail.com>
Authored: Thu Sep 10 10:22:31 2015 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Thu Sep 10 10:22:31 2015 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  11 +
 .../thrift/RetryingThriftCLIServiceClient.java  | 331 +++++++++++++++++++
 .../cli/TestRetryingThriftCLIServiceClient.java | 133 ++++++++
 3 files changed, 475 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9b11caff/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 8a00079..d2c5885 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2022,6 +2022,17 @@ public class HiveConf extends Configuration {
         "Session will be considered to be idle only if there is no activity, and there is no pending operation.\n" +
         " This setting takes effect only if session idle timeout (hive.server2.idle.session.timeout) and checking\n" +
         "(hive.server2.session.check.interval) are enabled."),
+    HIVE_SERVER2_THRIFT_CLIENT_RETRY_LIMIT("hive.server2.thrift.client.retry.limit", 1,"Number of retries upon " +
+      "failure of Thrift HiveServer2 calls"),
+    HIVE_SERVER2_THRIFT_CLIENT_CONNECTION_RETRY_LIMIT("hive.server2.thrift.client.connect.retry.limit", 1,"Number of " +
+      "retries while opening a connection to HiveServe2"),
+    HIVE_SERVER2_THRIFT_CLIENT_RETRY_DELAY_SECONDS("hive.server2.thrift.client.retry.delay.seconds", "1s",
+      new TimeValidator(TimeUnit.SECONDS), "Number of seconds for the HiveServer2 thrift client to wait between " +
+      "consecutive connection attempts. Also specifies the time to wait between retrying thrift calls upon failures"),
+    HIVE_SERVER2_THRIFT_CLIENT_USER("hive.server2.thrift.client.user", "anonymous","Username to use against thrift" +
+      " client"),
+    HIVE_SERVER2_THRIFT_CLIENT_PASSWORD("hive.server2.thrift.client.password", "anonymous","Password to use against " +
+      "thrift client"),
 
     HIVE_SECURITY_COMMAND_WHITELIST("hive.security.command.whitelist", "set,reset,dfs,add,list,delete,reload,compile",
         "Comma separated list of non-SQL Hive commands users are authorized to execute"),

http://git-wip-us.apache.org/repos/asf/hive/blob/9b11caff/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java b/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java
new file mode 100644
index 0000000..4bd7336
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java
@@ -0,0 +1,331 @@
+/**
+ * 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.hive.service.cli.thrift;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.service.auth.HiveAuthFactory;
+import org.apache.hive.service.auth.KerberosSaslHelper;
+import org.apache.hive.service.auth.PlainSaslHelper;
+import org.apache.hive.service.cli.*;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+
+import javax.security.sasl.SaslException;
+import java.lang.reflect.*;
+import java.net.SocketException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * RetryingThriftCLIServiceClient. Creates a proxy for a CLIServiceClient
+ * implementation and retries calls to it on failure.
+ */
+public class RetryingThriftCLIServiceClient implements InvocationHandler {
+  public static final Log LOG = LogFactory.getLog(RetryingThriftCLIServiceClient.class);
+  private ThriftCLIServiceClient base;
+  private final int retryLimit;
+  private final int retryDelaySeconds;
+  private HiveConf conf;
+  private TTransport transport;
+
+  public static class CLIServiceClientWrapper extends CLIServiceClient {
+    private final ICLIService cliService;
+
+    public CLIServiceClientWrapper(ICLIService icliService) {
+      cliService = icliService;
+    }
+
+    @Override
+    public SessionHandle openSession(String username, String password) throws HiveSQLException {
+      return cliService.openSession(username, password, Collections.<String, String>emptyMap());
+    }
+
+    @Override
+    public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, String owner,
+                                     String renewer) throws HiveSQLException {
+      return cliService.getDelegationToken(sessionHandle, authFactory, owner, renewer);
+    }
+
+    @Override
+    public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+                                      String tokenStr) throws HiveSQLException {
+      cliService.cancelDelegationToken(sessionHandle, authFactory, tokenStr);
+    }
+
+    @Override
+    public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+                                     String tokenStr) throws HiveSQLException {
+      cliService.renewDelegationToken(sessionHandle, authFactory, tokenStr);
+    }
+
+    @Override
+    public SessionHandle openSession(String username, String password, Map<String, String> configuration)
+      throws HiveSQLException {
+      return cliService.openSession(username, password, configuration);
+    }
+
+    @Override
+    public SessionHandle openSessionWithImpersonation(String username,
+                                                      String password,
+                                                      Map<String, String> configuration,
+                                                      String delegationToken) throws HiveSQLException {
+      return cliService.openSessionWithImpersonation(username, password, configuration, delegationToken);
+    }
+
+    @Override
+    public void closeSession(SessionHandle sessionHandle) throws HiveSQLException {
+      cliService.closeSession(sessionHandle);
+    }
+
+    @Override
+    public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType) throws HiveSQLException {
+      return cliService.getInfo(sessionHandle, getInfoType);
+    }
+
+    @Override
+    public OperationHandle executeStatement(SessionHandle sessionHandle,
+                                            String statement,
+                                            Map<String, String> confOverlay) throws HiveSQLException {
+      return cliService.executeStatement(sessionHandle, statement, confOverlay);
+    }
+
+    @Override
+    public OperationHandle executeStatementAsync(SessionHandle sessionHandle,
+                                                 String statement,
+                                                 Map<String, String> confOverlay) throws HiveSQLException {
+      return cliService.executeStatementAsync(sessionHandle, statement, confOverlay);
+    }
+
+    @Override
+    public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException {
+      return cliService.getTypeInfo(sessionHandle);
+    }
+
+    @Override
+    public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException {
+      return cliService.getCatalogs(sessionHandle);
+    }
+
+    @Override
+    public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName, String schemaName)
+      throws HiveSQLException {
+      return cliService.getSchemas(sessionHandle, catalogName, schemaName);
+    }
+
+    @Override
+    public OperationHandle getTables(SessionHandle sessionHandle, String catalogName, String schemaName,
+                                     String tableName, List<String> tableTypes) throws HiveSQLException {
+      return cliService.getTables(sessionHandle, catalogName, schemaName, tableName, tableTypes);
+    }
+
+    @Override
+    public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException {
+      return null;
+    }
+
+    @Override
+    public OperationHandle getColumns(SessionHandle sessionHandle, String catalogName, String schemaName,
+                                      String tableName, String columnName) throws HiveSQLException {
+      return cliService.getColumns(sessionHandle, catalogName, schemaName, tableName, columnName);
+    }
+
+    @Override
+    public OperationHandle getFunctions(SessionHandle sessionHandle, String catalogName, String schemaName,
+                                        String functionName) throws HiveSQLException {
+      return cliService.getFunctions(sessionHandle, catalogName, schemaName, functionName);
+    }
+
+    @Override
+    public OperationStatus getOperationStatus(OperationHandle opHandle) throws HiveSQLException {
+      return cliService.getOperationStatus(opHandle);
+    }
+
+    @Override
+    public void cancelOperation(OperationHandle opHandle) throws HiveSQLException {
+      cliService.cancelOperation(opHandle);
+    }
+
+    @Override
+    public void closeOperation(OperationHandle opHandle) throws HiveSQLException {
+      cliService.closeOperation(opHandle);
+    }
+
+    @Override
+    public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException {
+      return cliService.getResultSetMetadata(opHandle);
+    }
+
+    @Override
+    public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, long maxRows,
+                               FetchType fetchType) throws HiveSQLException {
+      return cliService.fetchResults(opHandle, orientation, maxRows, fetchType);
+    }
+  }
+
+  protected RetryingThriftCLIServiceClient(HiveConf conf) {
+    this.conf = conf;
+    retryLimit = conf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_RETRY_LIMIT);
+    retryDelaySeconds = (int) conf.getTimeVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_RETRY_DELAY_SECONDS,
+      TimeUnit.SECONDS);
+  }
+
+  public static CLIServiceClient newRetryingCLIServiceClient(HiveConf conf) throws HiveSQLException {
+    RetryingThriftCLIServiceClient retryClient = new RetryingThriftCLIServiceClient(conf);
+    retryClient.connectWithRetry(conf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_CONNECTION_RETRY_LIMIT));
+    ICLIService cliService =
+      (ICLIService) Proxy.newProxyInstance(RetryingThriftCLIServiceClient.class.getClassLoader(),
+        CLIServiceClient.class.getInterfaces(), retryClient);
+    return new CLIServiceClientWrapper(cliService);
+  }
+
+  protected void connectWithRetry(int retries) throws HiveSQLException {
+    for (int i = 0 ; i < retries; i++) {
+      try {
+        connect(conf);
+        break;
+      } catch (TTransportException e) {
+        if (i + 1 == retries) {
+          throw new HiveSQLException("Unable to connect after " + retries + " retries", e);
+        }
+        LOG.warn("Connection attempt " + i, e);
+      }
+      try {
+        Thread.sleep(retryDelaySeconds * 1000);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted", e);
+      }
+    }
+  }
+
+  protected synchronized TTransport connect(HiveConf conf) throws HiveSQLException, TTransportException {
+    if (transport != null && transport.isOpen()) {
+      transport.close();
+    }
+
+    String host = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST);
+    int port = conf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT);
+    LOG.info("Connecting to " + host + ":" + port);
+
+    transport = new TSocket(host, port);
+    ((TSocket) transport).setTimeout((int) conf.getTimeVar(HiveConf.ConfVars.SERVER_READ_SOCKET_TIMEOUT,
+      TimeUnit.SECONDS) * 1000);
+    try {
+      ((TSocket) transport).getSocket().setKeepAlive(conf.getBoolVar(HiveConf.ConfVars.SERVER_TCP_KEEP_ALIVE));
+    } catch (SocketException e) {
+      LOG.error("Error setting keep alive to " + conf.getBoolVar(HiveConf.ConfVars.SERVER_TCP_KEEP_ALIVE), e);
+    }
+
+    String userName = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_USER);
+    String passwd = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_PASSWORD);
+
+    try {
+      transport = PlainSaslHelper.getPlainTransport(userName, passwd, transport);
+    } catch (SaslException e) {
+      LOG.error("Error creating plain SASL transport", e);
+    }
+
+    TProtocol protocol = new TBinaryProtocol(transport);
+    transport.open();
+    base = new ThriftCLIServiceClient(new TCLIService.Client(protocol));
+    LOG.info("Connected!");
+    return transport;
+  }
+
+  protected class InvocationResult {
+    final boolean success;
+    final Object result;
+    final Throwable exception;
+
+    InvocationResult(boolean success, Object result, Throwable exception) {
+      this.success = success;
+      this.result = result;
+      this.exception = exception;
+    }
+  }
+
+  protected InvocationResult invokeInternal(Method method, Object[] args) throws Throwable {
+    InvocationResult result;
+    try {
+      Object methodResult = method.invoke(base, args);
+      result = new InvocationResult(true, methodResult, null);
+    } catch (UndeclaredThrowableException e) {
+      throw e.getCause();
+    } catch (InvocationTargetException e) {
+      if (e.getCause() instanceof HiveSQLException) {
+        HiveSQLException hiveExc = (HiveSQLException) e.getCause();
+        Throwable cause = hiveExc.getCause();
+        if ((cause instanceof TApplicationException) ||
+          (cause instanceof TProtocolException) ||
+          (cause instanceof TTransportException)) {
+          result =  new InvocationResult(false, null, hiveExc);
+        } else {
+          throw hiveExc;
+        }
+      } else {
+        throw e.getCause();
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public Object invoke(Object o, Method method, Object[] args) throws Throwable {
+    int attempts = 0;
+
+    while (true) {
+      attempts++;
+      InvocationResult invokeResult = invokeInternal(method, args);
+      if (invokeResult.success) {
+        return invokeResult.result;
+      }
+
+      // Error because of thrift client, we have to recreate base object
+      connectWithRetry(conf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_CONNECTION_RETRY_LIMIT));
+
+      if (attempts >=  retryLimit) {
+        LOG.error(method.getName() + " failed after " + attempts + " retries.",  invokeResult.exception);
+        throw invokeResult.exception;
+      }
+
+      LOG.warn("Last call ThriftCLIServiceClient." + method.getName() + " failed, attempts = " + attempts,
+        invokeResult.exception);
+      Thread.sleep(retryDelaySeconds * 1000);
+    }
+  }
+
+  public int getRetryLimit() {
+    return retryLimit;
+  }
+
+  public int getRetryDelaySeconds() {
+    return retryDelaySeconds;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9b11caff/service/src/test/org/apache/hive/service/cli/TestRetryingThriftCLIServiceClient.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/TestRetryingThriftCLIServiceClient.java b/service/src/test/org/apache/hive/service/cli/TestRetryingThriftCLIServiceClient.java
new file mode 100644
index 0000000..3798053
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/cli/TestRetryingThriftCLIServiceClient.java
@@ -0,0 +1,133 @@
+/**
+ * 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.hive.service.cli;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.service.auth.HiveAuthFactory;
+import org.apache.hive.service.cli.thrift.RetryingThriftCLIServiceClient;
+import org.apache.hive.service.cli.thrift.ThriftCLIService;
+import org.apache.hive.service.server.HiveServer2;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.junit.Test;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test CLI service with a retrying client. All tests should pass. This is to validate that calls
+ * are transferred successfully.
+ */
+public class TestRetryingThriftCLIServiceClient {
+  protected static ThriftCLIService service;
+
+  static class RetryingThriftCLIServiceClientTest extends RetryingThriftCLIServiceClient {
+    int callCount = 0;
+    int connectCount = 0;
+    static RetryingThriftCLIServiceClientTest handlerInst;
+
+    protected RetryingThriftCLIServiceClientTest(HiveConf conf) {
+      super(conf);
+    }
+
+    public static CLIServiceClient newRetryingCLIServiceClient(HiveConf conf) throws HiveSQLException {
+      handlerInst = new RetryingThriftCLIServiceClientTest(conf);
+      handlerInst.connectWithRetry(conf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_RETRY_LIMIT));
+
+      ICLIService cliService =
+        (ICLIService) Proxy.newProxyInstance(RetryingThriftCLIServiceClientTest.class.getClassLoader(),
+          CLIServiceClient.class.getInterfaces(), handlerInst);
+      return new CLIServiceClientWrapper(cliService);
+    }
+
+    @Override
+    protected InvocationResult invokeInternal(Method method, Object[] args) throws Throwable {
+      System.out.println("## Calling: " + method.getName() + ", " + callCount + "/" + getRetryLimit());
+      callCount++;
+      return super.invokeInternal(method, args);
+    }
+
+    @Override
+    protected synchronized TTransport connect(HiveConf conf) throws HiveSQLException, TTransportException {
+      connectCount++;
+      return super.connect(conf);
+    }
+  }
+  @Test
+  public void testRetryBehaviour() throws Exception {
+    // Start hive server2
+    HiveConf hiveConf = new HiveConf();
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, "localhost");
+    hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT, 15000);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthFactory.AuthTypes.NONE.toString());
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE, "binary");
+    hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_RETRY_LIMIT, 3);
+    hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_CONNECTION_RETRY_LIMIT, 3);
+    hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS, 10);
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT, "1s");
+
+    final HiveServer2 server = new HiveServer2();
+    server.init(hiveConf);
+    server.start();
+    Thread.sleep(5000);
+    System.out.println("## HiveServer started");
+
+    // Check if giving invalid address causes retry in connection attempt
+    hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT, 17000);
+    try {
+      CLIServiceClient cliServiceClient =
+        RetryingThriftCLIServiceClientTest.newRetryingCLIServiceClient(hiveConf);
+      fail("Expected to throw exception for invalid port");
+    } catch (HiveSQLException sqlExc) {
+      assertTrue(sqlExc.getCause() instanceof TTransportException);
+      assertTrue(sqlExc.getMessage().contains("3"));
+    }
+
+    // Reset port setting
+    hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT, 15000);
+    // Create client
+    CLIServiceClient cliServiceClient =
+      RetryingThriftCLIServiceClientTest.newRetryingCLIServiceClient(hiveConf);
+    System.out.println("## Created client");
+
+    // kill server
+    server.stop();
+    Thread.sleep(5000);
+
+    // submit few queries
+    try {
+      Map<String, String> confOverlay = new HashMap<String, String>();
+      RetryingThriftCLIServiceClientTest.handlerInst.callCount = 0;
+      RetryingThriftCLIServiceClientTest.handlerInst.connectCount = 0;
+      SessionHandle session = cliServiceClient.openSession("anonymous", "anonymous");
+    } catch (HiveSQLException exc) {
+      exc.printStackTrace();
+      assertTrue(exc.getCause() instanceof TException);
+      assertEquals(1, RetryingThriftCLIServiceClientTest.handlerInst.callCount);
+      assertEquals(3, RetryingThriftCLIServiceClientTest.handlerInst.connectCount);
+    }
+
+  }
+}


[37/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorized_parquet.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorized_parquet.q.out b/ql/src/test/results/clientpositive/tez/vectorized_parquet.q.out
index 8118139..966f131 100644
--- a/ql/src/test/results/clientpositive/tez/vectorized_parquet.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorized_parquet.q.out
@@ -135,11 +135,11 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 73728 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint), cint (type: int), csmallint (type: smallint), cstring1 (type: string), cfloat (type: float), cdouble (type: double)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                    outputColumnNames: ctinyint, cint, csmallint, cstring1, cfloat, cdouble
                     Statistics: Num rows: 12288 Data size: 73728 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col1), min(_col2), count(_col3), avg(_col4), stddev_pop(_col5)
-                      keys: _col0 (type: tinyint)
+                      aggregations: max(cint), min(csmallint), count(cstring1), avg(cfloat), stddev_pop(cdouble)
+                      keys: ctinyint (type: tinyint)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                       Statistics: Num rows: 12288 Data size: 73728 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorized_timestamp_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorized_timestamp_funcs.q.out b/ql/src/test/results/clientpositive/tez/vectorized_timestamp_funcs.q.out
index b285a4f..cb47605 100644
--- a/ql/src/test/results/clientpositive/tez/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorized_timestamp_funcs.q.out
@@ -644,10 +644,10 @@ STAGE PLANS:
                   Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctimestamp1 (type: timestamp)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctimestamp1
                     Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0), max(_col0), count(_col0), count()
+                      aggregations: min(ctimestamp1), max(ctimestamp1), count(ctimestamp1), count()
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: NONE
@@ -725,10 +725,10 @@ STAGE PLANS:
                   Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctimestamp1 (type: timestamp)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctimestamp1
                     Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(ctimestamp1)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -815,10 +815,10 @@ STAGE PLANS:
                   Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctimestamp1 (type: timestamp)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctimestamp1
                     Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: avg(_col0), variance(_col0), var_pop(_col0), var_samp(_col0), std(_col0), stddev(_col0), stddev_pop(_col0), stddev_samp(_col0)
+                      aggregations: avg(ctimestamp1), variance(ctimestamp1), var_pop(ctimestamp1), var_samp(ctimestamp1), std(ctimestamp1), stddev(ctimestamp1), stddev_pop(ctimestamp1), stddev_samp(ctimestamp1)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/udf8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf8.q.out b/ql/src/test/results/clientpositive/udf8.q.out
index 8276ba9..1f43e23 100644
--- a/ql/src/test/results/clientpositive/udf8.q.out
+++ b/ql/src/test/results/clientpositive/udf8.q.out
@@ -43,10 +43,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: c1 (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: c1
               Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: avg(_col0), sum(_col0), count(_col0)
+                aggregations: avg(c1), sum(c1), count(c1)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/udf_count.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_count.q.out b/ql/src/test/results/clientpositive/udf_count.q.out
index c9a596f..430893d 100644
--- a/ql/src/test/results/clientpositive/udf_count.q.out
+++ b/ql/src/test/results/clientpositive/udf_count.q.out
@@ -29,10 +29,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -86,11 +86,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col0)
-                keys: _col0 (type: string)
+                aggregations: count(DISTINCT key)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -144,11 +144,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col0, _col1)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT key, value)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/union11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union11.q.out b/ql/src/test/results/clientpositive/union11.q.out
index 0d0b8fa..55d593d 100644
--- a/ql/src/test/results/clientpositive/union11.q.out
+++ b/ql/src/test/results/clientpositive/union11.q.out
@@ -31,28 +31,30 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              expressions: key (type: string)
+              outputColumnNames: key
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(1)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: 'tst1' (type: string)
             outputColumnNames: _col0
-            Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -65,48 +67,48 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Union
-              Statistics: Num rows: 3 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count(1)
                 keys: _col0 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: bigint)
           TableScan
             Union
-              Statistics: Num rows: 3 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count(1)
                 keys: _col0 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: bigint)
           TableScan
             Union
-              Statistics: Num rows: 3 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count(1)
                 keys: _col0 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -114,10 +116,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -128,28 +130,30 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              expressions: key (type: string)
+              outputColumnNames: key
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(1)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: 'tst2' (type: string)
             outputColumnNames: _col0
-            Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -162,28 +166,30 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              expressions: key (type: string)
+              outputColumnNames: key
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(1)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: 'tst3' (type: string)
             outputColumnNames: _col0
-            Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/union14.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union14.q.out b/ql/src/test/results/clientpositive/union14.q.out
index 7a7e938..eabdaa8 100644
--- a/ql/src/test/results/clientpositive/union14.q.out
+++ b/ql/src/test/results/clientpositive/union14.q.out
@@ -27,28 +27,30 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              expressions: key (type: string)
+              outputColumnNames: key
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(1)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: 'tst1' (type: string)
             outputColumnNames: _col0
-            Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -67,33 +69,33 @@ STAGE PLANS:
               outputColumnNames: _col0
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 26 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   aggregations: count(1)
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col1 (type: bigint)
           TableScan
             Union
-              Statistics: Num rows: 26 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count(1)
                 keys: _col0 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -101,10 +103,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/union15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union15.q.out b/ql/src/test/results/clientpositive/union15.q.out
index 148f25f..be7f966 100644
--- a/ql/src/test/results/clientpositive/union15.q.out
+++ b/ql/src/test/results/clientpositive/union15.q.out
@@ -29,28 +29,30 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              expressions: key (type: string)
+              outputColumnNames: key
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(1)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: 'tst1' (type: string)
             outputColumnNames: _col0
-            Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -63,18 +65,18 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Union
-              Statistics: Num rows: 51 Data size: 470 Basic stats: COMPLETE Column stats: PARTIAL
+              Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count(1)
                 keys: _col0 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: bigint)
           TableScan
             alias: s2
@@ -84,18 +86,18 @@ STAGE PLANS:
               outputColumnNames: _col0
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 51 Data size: 470 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   aggregations: count(1)
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col1 (type: bigint)
           TableScan
             alias: s2
@@ -105,18 +107,18 @@ STAGE PLANS:
               outputColumnNames: _col0
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 51 Data size: 470 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   aggregations: count(1)
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -124,10 +126,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/union28.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union28.q.out b/ql/src/test/results/clientpositive/union28.q.out
index 38f8ee6..c3789d0 100644
--- a/ql/src/test/results/clientpositive/union28.q.out
+++ b/ql/src/test/results/clientpositive/union28.q.out
@@ -53,10 +53,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -193,10 +193,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/union30.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union30.q.out b/ql/src/test/results/clientpositive/union30.q.out
index 894ed49..26a27c8 100644
--- a/ql/src/test/results/clientpositive/union30.q.out
+++ b/ql/src/test/results/clientpositive/union30.q.out
@@ -67,10 +67,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -228,10 +228,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/union33.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union33.q.out b/ql/src/test/results/clientpositive/union33.q.out
index 308cd8b..17e0844 100644
--- a/ql/src/test/results/clientpositive/union33.q.out
+++ b/ql/src/test/results/clientpositive/union33.q.out
@@ -53,11 +53,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -268,11 +268,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/union5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union5.q.out b/ql/src/test/results/clientpositive/union5.q.out
index 75389f8..f57c60a 100644
--- a/ql/src/test/results/clientpositive/union5.q.out
+++ b/ql/src/test/results/clientpositive/union5.q.out
@@ -26,28 +26,30 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              expressions: key (type: string)
+              outputColumnNames: key
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(1)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: 'tst1' (type: string)
             outputColumnNames: _col0
-            Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -60,33 +62,33 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Union
-              Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count(1)
                 keys: _col0 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: bigint)
           TableScan
             Union
-              Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count(1)
                 keys: _col0 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -94,10 +96,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -108,28 +110,30 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              expressions: key (type: string)
+              outputColumnNames: key
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(1)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: 'tst2' (type: string)
             outputColumnNames: _col0
-            Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/union7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union7.q.out b/ql/src/test/results/clientpositive/union7.q.out
index 592f073..162fe36 100644
--- a/ql/src/test/results/clientpositive/union7.q.out
+++ b/ql/src/test/results/clientpositive/union7.q.out
@@ -25,28 +25,30 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              expressions: key (type: string)
+              outputColumnNames: key
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(1)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: 'tst1' (type: string)
             outputColumnNames: _col0
-            Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -59,18 +61,18 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Union
-              Statistics: Num rows: 26 Data size: 279 Basic stats: COMPLETE Column stats: PARTIAL
+              Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count(1)
                 keys: _col0 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: bigint)
           TableScan
             alias: s2
@@ -80,18 +82,18 @@ STAGE PLANS:
               outputColumnNames: _col0
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 26 Data size: 279 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   aggregations: count(1)
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -99,10 +101,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+            Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/unionDistinct_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/unionDistinct_1.q.out b/ql/src/test/results/clientpositive/unionDistinct_1.q.out
index 8d74fbe..81c46da 100644
--- a/ql/src/test/results/clientpositive/unionDistinct_1.q.out
+++ b/ql/src/test/results/clientpositive/unionDistinct_1.q.out
@@ -15008,11 +15008,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -15172,11 +15172,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/union_remove_21.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_21.q.out b/ql/src/test/results/clientpositive/union_remove_21.q.out
index 4743d8d..c956940 100644
--- a/ql/src/test/results/clientpositive/union_remove_21.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_21.q.out
@@ -76,10 +76,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
@@ -121,10 +121,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_aggregate_9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_aggregate_9.q.out b/ql/src/test/results/clientpositive/vector_aggregate_9.q.out
index f81816c..72dc004 100644
--- a/ql/src/test/results/clientpositive/vector_aggregate_9.q.out
+++ b/ql/src/test/results/clientpositive/vector_aggregate_9.q.out
@@ -120,10 +120,10 @@ STAGE PLANS:
             Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: dc (type: decimal(38,18))
-              outputColumnNames: _col0
+              outputColumnNames: dc
               Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: min(_col0), max(_col0), sum(_col0), avg(_col0)
+                aggregations: min(dc), max(dc), sum(dc), avg(dc)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_aggregate_without_gby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_aggregate_without_gby.q.out b/ql/src/test/results/clientpositive/vector_aggregate_without_gby.q.out
index 1175cb8..cd4e4cc 100644
--- a/ql/src/test/results/clientpositive/vector_aggregate_without_gby.q.out
+++ b/ql/src/test/results/clientpositive/vector_aggregate_without_gby.q.out
@@ -53,10 +53,10 @@ STAGE PLANS:
               Statistics: Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: dt (type: int), greg_dt (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: dt, greg_dt
                 Statistics: Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: max(_col0), max(_col1)
+                  aggregations: max(dt), max(greg_dt)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out b/ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out
index c3e4d52..7da8ae0 100644
--- a/ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out
+++ b/ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out
@@ -213,11 +213,11 @@ STAGE PLANS:
             Statistics: Num rows: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: bin (type: binary)
-              outputColumnNames: _col0
+              outputColumnNames: bin
               Statistics: Num rows: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: binary)
+                keys: bin (type: binary)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_count_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_count_distinct.q.out b/ql/src/test/results/clientpositive/vector_count_distinct.q.out
index b87b2c7..0d491b3 100644
--- a/ql/src/test/results/clientpositive/vector_count_distinct.q.out
+++ b/ql/src/test/results/clientpositive/vector_count_distinct.q.out
@@ -1252,11 +1252,11 @@ STAGE PLANS:
             Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ws_order_number (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: ws_order_number
               Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col0)
-                keys: _col0 (type: int)
+                aggregations: count(DISTINCT ws_order_number)
+                keys: ws_order_number (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out b/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out
index 02c1c83..cf975d1 100644
--- a/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out
+++ b/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out
@@ -49,11 +49,11 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: cint (type: int), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14))
-              outputColumnNames: _col0, _col1, _col2
+              outputColumnNames: cint, cdecimal1, cdecimal2
               Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1), max(_col1), min(_col1), sum(_col1), count(_col2), max(_col2), min(_col2), sum(_col2), count()
-                keys: _col0 (type: int)
+                aggregations: count(cdecimal1), max(cdecimal1), min(cdecimal1), sum(cdecimal1), count(cdecimal2), max(cdecimal2), min(cdecimal2), sum(cdecimal2), count()
+                keys: cint (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                 Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
@@ -147,11 +147,11 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: cint (type: int), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14))
-              outputColumnNames: _col0, _col1, _col2
+              outputColumnNames: cint, cdecimal1, cdecimal2
               Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1), max(_col1), min(_col1), sum(_col1), avg(_col1), stddev_pop(_col1), stddev_samp(_col1), count(_col2), max(_col2), min(_col2), sum(_col2), avg(_col2), stddev_pop(_col2), stddev_samp(_col2), count()
-                keys: _col0 (type: int)
+                aggregations: count(cdecimal1), max(cdecimal1), min(cdecimal1), sum(cdecimal1), avg(cdecimal1), stddev_pop(cdecimal1), stddev_samp(cdecimal1), count(cdecimal2), max(cdecimal2), min(cdecimal2), sum(cdecimal2), avg(cdecimal2), stddev_pop(cdecimal2), stddev_samp(cdecimal2), count()
+                keys: cint (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15
                 Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_decimal_precision.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_decimal_precision.q.out b/ql/src/test/results/clientpositive/vector_decimal_precision.q.out
index 16f2e3f..f2aaf8d 100644
--- a/ql/src/test/results/clientpositive/vector_decimal_precision.q.out
+++ b/ql/src/test/results/clientpositive/vector_decimal_precision.q.out
@@ -562,10 +562,10 @@ STAGE PLANS:
             Statistics: Num rows: 75 Data size: 3472 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: dec (type: decimal(20,10))
-              outputColumnNames: _col0
+              outputColumnNames: dec
               Statistics: Num rows: 75 Data size: 3472 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: avg(_col0), sum(_col0)
+                aggregations: avg(dec), sum(dec)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_decimal_udf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_decimal_udf.q.out b/ql/src/test/results/clientpositive/vector_decimal_udf.q.out
index 91a585c..cfd2a55 100644
--- a/ql/src/test/results/clientpositive/vector_decimal_udf.q.out
+++ b/ql/src/test/results/clientpositive/vector_decimal_udf.q.out
@@ -1574,11 +1574,11 @@ STAGE PLANS:
             Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: value (type: int), key (type: decimal(20,10))
-              outputColumnNames: _col0, _col1
+              outputColumnNames: value, key
               Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col1), count(_col1), avg(_col1)
-                keys: _col0 (type: int)
+                aggregations: sum(key), count(key), avg(key)
+                keys: value (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
@@ -2239,11 +2239,11 @@ STAGE PLANS:
             Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: value (type: int), key (type: decimal(20,10))
-              outputColumnNames: _col0, _col1
+              outputColumnNames: value, key
               Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: stddev(_col1), variance(_col1)
-                keys: _col0 (type: int)
+                aggregations: stddev(key), variance(key)
+                keys: value (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
@@ -2319,11 +2319,11 @@ STAGE PLANS:
             Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: value (type: int), key (type: decimal(20,10))
-              outputColumnNames: _col0, _col1
+              outputColumnNames: value, key
               Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: stddev_samp(_col1), var_samp(_col1)
-                keys: _col0 (type: int)
+                aggregations: stddev_samp(key), var_samp(key)
+                keys: value (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
@@ -2458,10 +2458,10 @@ STAGE PLANS:
             Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: decimal(20,10))
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: min(_col0)
+                aggregations: min(key)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
@@ -2518,10 +2518,10 @@ STAGE PLANS:
             Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: decimal(20,10))
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: max(_col0)
+                aggregations: max(key)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
@@ -2578,10 +2578,10 @@ STAGE PLANS:
             Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: decimal(20,10))
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 38 Data size: 4296 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(key)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_distinct_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_distinct_2.q.out b/ql/src/test/results/clientpositive/vector_distinct_2.q.out
index 4153155..6b552be 100644
--- a/ql/src/test/results/clientpositive/vector_distinct_2.q.out
+++ b/ql/src/test/results/clientpositive/vector_distinct_2.q.out
@@ -123,33 +123,37 @@ STAGE PLANS:
             alias: vectortab2korc
             Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: s (type: string), t (type: tinyint)
-              outputColumnNames: _col0, _col1
+              expressions: t (type: tinyint), s (type: string)
+              outputColumnNames: t, s
               Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: tinyint)
+                keys: t (type: tinyint), s (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col0 (type: string), _col1 (type: tinyint)
+                  key expressions: _col0 (type: tinyint), _col1 (type: string)
                   sort order: ++
-                  Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
+                  Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: string)
                   Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
-          keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
+          keys: KEY._col0 (type: tinyint), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
+          Select Operator
+            expressions: _col1 (type: string), _col0 (type: tinyint)
+            outputColumnNames: _col0, _col1
             Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-            table:
-                input format: org.apache.hadoop.mapred.TextInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_groupby_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_groupby_3.q.out b/ql/src/test/results/clientpositive/vector_groupby_3.q.out
index af7cb58..ffe42a7 100644
--- a/ql/src/test/results/clientpositive/vector_groupby_3.q.out
+++ b/ql/src/test/results/clientpositive/vector_groupby_3.q.out
@@ -123,36 +123,40 @@ STAGE PLANS:
             alias: vectortab2korc
             Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: s (type: string), t (type: tinyint), b (type: bigint)
-              outputColumnNames: _col0, _col1, _col2
+              expressions: t (type: tinyint), s (type: string), b (type: bigint)
+              outputColumnNames: t, s, b
               Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: max(_col2)
-                keys: _col0 (type: string), _col1 (type: tinyint)
+                aggregations: max(b)
+                keys: t (type: tinyint), s (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col0 (type: string), _col1 (type: tinyint)
+                  key expressions: _col0 (type: tinyint), _col1 (type: string)
                   sort order: ++
-                  Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
+                  Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: string)
                   Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col2 (type: bigint)
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           aggregations: max(VALUE._col0)
-          keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
+          keys: KEY._col0 (type: tinyint), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
+          Select Operator
+            expressions: _col1 (type: string), _col0 (type: tinyint), _col2 (type: bigint)
+            outputColumnNames: _col0, _col1, _col2
             Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-            table:
-                input format: org.apache.hadoop.mapred.TextInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_groupby_reduce.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_groupby_reduce.q.out b/ql/src/test/results/clientpositive/vector_groupby_reduce.q.out
index edcb0b3..331ba4f 100644
--- a/ql/src/test/results/clientpositive/vector_groupby_reduce.q.out
+++ b/ql/src/test/results/clientpositive/vector_groupby_reduce.q.out
@@ -240,10 +240,10 @@ STAGE PLANS:
             Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ss_ticket_number (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: ss_ticket_number
               Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: int)
+                keys: ss_ticket_number (type: int)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE
@@ -357,10 +357,10 @@ STAGE PLANS:
             Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ss_ticket_number (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: ss_ticket_number
               Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: int)
+                keys: ss_ticket_number (type: int)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_grouping_sets.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_grouping_sets.q.out b/ql/src/test/results/clientpositive/vector_grouping_sets.q.out
index 69d6187..32e95c0 100644
--- a/ql/src/test/results/clientpositive/vector_grouping_sets.q.out
+++ b/ql/src/test/results/clientpositive/vector_grouping_sets.q.out
@@ -121,10 +121,10 @@ STAGE PLANS:
             Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: s_store_id (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: s_store_id
               Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), '0' (type: string)
+                keys: s_store_id (type: string), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE
@@ -196,10 +196,10 @@ STAGE PLANS:
             Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: s_store_id (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: s_store_id
               Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), '0' (type: string)
+                keys: s_store_id (type: string), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_left_outer_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_left_outer_join.q.out b/ql/src/test/results/clientpositive/vector_left_outer_join.q.out
index 2f986c2..2f8efd5 100644
--- a/ql/src/test/results/clientpositive/vector_left_outer_join.q.out
+++ b/ql/src/test/results/clientpositive/vector_left_outer_join.q.out
@@ -25,14 +25,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           TableScan
             alias: c
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -44,7 +44,7 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: int)
                   1 _col0 (type: int)
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           TableScan
             alias: c
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out b/ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out
index fdd7ea8..ee74fbe 100644
--- a/ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out
+++ b/ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out
@@ -44,20 +44,16 @@ STAGE PLANS:
             Filter Operator
               predicate: l_partkey is not null (type: boolean)
               Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: l_partkey (type: int)
+              Group By Operator
+                keys: l_partkey (type: int)
+                mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: int)
-                  mode: hash
-                  outputColumnNames: _col0
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
-                    Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: int)
@@ -315,20 +311,16 @@ STAGE PLANS:
             Filter Operator
               predicate: l_partkey is not null (type: boolean)
               Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: l_partkey (type: int)
+              Group By Operator
+                keys: l_partkey (type: int)
+                mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: int)
-                  mode: hash
-                  outputColumnNames: _col0
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
-                    Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: int)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_orderby_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_orderby_5.q.out b/ql/src/test/results/clientpositive/vector_orderby_5.q.out
index 45ccc62..a3be243 100644
--- a/ql/src/test/results/clientpositive/vector_orderby_5.q.out
+++ b/ql/src/test/results/clientpositive/vector_orderby_5.q.out
@@ -121,11 +121,11 @@ STAGE PLANS:
             Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: bo (type: boolean), b (type: bigint)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: bo, b
               Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: max(_col1)
-                keys: _col0 (type: boolean)
+                aggregations: max(b)
+                keys: bo (type: boolean)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_outer_join1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_outer_join1.q.out b/ql/src/test/results/clientpositive/vector_outer_join1.q.out
index 8e8f7fa..fd86093 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join1.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join1.q.out
@@ -489,14 +489,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           TableScan
             alias: c
             Statistics: Num rows: 15 Data size: 3320 Basic stats: COMPLETE Column stats: NONE
@@ -508,7 +508,7 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: int)
                   1 _col0 (type: int)
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           TableScan
             alias: c
             Statistics: Num rows: 15 Data size: 3320 Basic stats: COMPLETE Column stats: NONE


[02/50] [abbrv] hive git commit: HIVE-11696: Exception when table-level serde is Parquet while partition-level serde is JSON (Aihua Xu, reviewed by Chao Sun)

Posted by xu...@apache.org.
HIVE-11696: Exception when table-level serde is Parquet while partition-level serde is JSON (Aihua Xu, reviewed by Chao Sun)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d94c0f65
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d94c0f65
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d94c0f65

Branch: refs/heads/spark
Commit: d94c0f65d85d1a0c6b31a75ecf1d1b805d823a32
Parents: d51c62a
Author: Chao Sun <su...@apache.org>
Authored: Wed Sep 9 12:56:24 2015 -0700
Committer: Chao Sun <su...@apache.org>
Committed: Wed Sep 9 12:56:45 2015 -0700

----------------------------------------------------------------------
 .../serde/ParquetHiveArrayInspector.java        |  12 +
 .../parquet_mixed_partition_formats.q           |  42 +++
 .../parquet_mixed_partition_formats.q.out       | 303 +++++++++++++++++++
 3 files changed, 357 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d94c0f65/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveArrayInspector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveArrayInspector.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveArrayInspector.java
index bde0dcb..05e92b5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveArrayInspector.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveArrayInspector.java
@@ -68,6 +68,10 @@ public class ParquetHiveArrayInspector implements SettableListObjectInspector {
       }
     }
 
+    if (data instanceof List) {
+      return ((List)data).get(index);
+    }
+
     throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
   }
 
@@ -86,6 +90,10 @@ public class ParquetHiveArrayInspector implements SettableListObjectInspector {
       return array.length;
     }
 
+    if (data instanceof List) {
+      return ((List)data).size();
+    }
+
     throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
   }
 
@@ -109,6 +117,10 @@ public class ParquetHiveArrayInspector implements SettableListObjectInspector {
       return list;
     }
 
+    if (data instanceof List) {
+      return (List<?>)data;
+    }
+
     throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d94c0f65/ql/src/test/queries/clientpositive/parquet_mixed_partition_formats.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_mixed_partition_formats.q b/ql/src/test/queries/clientpositive/parquet_mixed_partition_formats.q
new file mode 100644
index 0000000..4d7d088
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/parquet_mixed_partition_formats.q
@@ -0,0 +1,42 @@
+DROP TABLE if exists parquet_mixed_partition_formats;
+
+CREATE TABLE parquet_mixed_partition_formats (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary string,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date)
+PARTITIONED BY (dateint int)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':';
+
+---- partition dateint=20140330 is stored as TEXTFILE
+LOAD DATA LOCAL INPATH '../../data/files/parquet_types.txt' OVERWRITE INTO TABLE parquet_mixed_partition_formats PARTITION (dateint=20140330);
+
+SELECT * FROM parquet_mixed_partition_formats;
+
+DESCRIBE FORMATTED parquet_mixed_partition_formats PARTITION (dateint=20140330);
+
+---change table serde and file format to PARQUET----
+
+ALTER TABLE parquet_mixed_partition_formats
+     SET FILEFORMAT
+     INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+     OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+     SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe';
+
+DESCRIBE FORMATTED parquet_mixed_partition_formats;
+DESCRIBE FORMATTED parquet_mixed_partition_formats PARTITION (dateint=20140330);
+
+SELECT * FROM parquet_mixed_partition_formats;

http://git-wip-us.apache.org/repos/asf/hive/blob/d94c0f65/ql/src/test/results/clientpositive/parquet_mixed_partition_formats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_mixed_partition_formats.q.out b/ql/src/test/results/clientpositive/parquet_mixed_partition_formats.q.out
new file mode 100644
index 0000000..a412350
--- /dev/null
+++ b/ql/src/test/results/clientpositive/parquet_mixed_partition_formats.q.out
@@ -0,0 +1,303 @@
+PREHOOK: query: DROP TABLE if exists parquet_mixed_partition_formats
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE if exists parquet_mixed_partition_formats
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE parquet_mixed_partition_formats (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary string,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date)
+PARTITIONED BY (dateint int)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':'
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_mixed_partition_formats
+POSTHOOK: query: CREATE TABLE parquet_mixed_partition_formats (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary string,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date)
+PARTITIONED BY (dateint int)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_mixed_partition_formats
+PREHOOK: query: ---- partition dateint=20140330 is stored as TEXTFILE
+LOAD DATA LOCAL INPATH '../../data/files/parquet_types.txt' OVERWRITE INTO TABLE parquet_mixed_partition_formats PARTITION (dateint=20140330)
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@parquet_mixed_partition_formats
+POSTHOOK: query: ---- partition dateint=20140330 is stored as TEXTFILE
+LOAD DATA LOCAL INPATH '../../data/files/parquet_types.txt' OVERWRITE INTO TABLE parquet_mixed_partition_formats PARTITION (dateint=20140330)
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@parquet_mixed_partition_formats
+POSTHOOK: Output: default@parquet_mixed_partition_formats@dateint=20140330
+PREHOOK: query: SELECT * FROM parquet_mixed_partition_formats
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_mixed_partition_formats
+PREHOOK: Input: default@parquet_mixed_partition_formats@dateint=20140330
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM parquet_mixed_partition_formats
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_mixed_partition_formats
+POSTHOOK: Input: default@parquet_mixed_partition_formats@dateint=20140330
+#### A masked pattern was here ####
+100	1	1	1.0	0.0	abc	2011-01-01 01:01:01.111111111	a    	a  	B4F3CAFDBEDD	{"k1":"v1"}	[101,200]	{"c1":10,"c2":"a"}	2011-01-01	20140330
+101	2	2	1.1	0.3	def	2012-02-02 02:02:02.222222222	ab   	ab 	68692CCAC0BDE7	{"k2":"v2"}	[102,200]	{"c1":10,"c2":"d"}	2012-02-02	20140330
+102	3	3	1.2	0.6	ghi	2013-03-03 03:03:03.333333333	abc  	abc	B4F3CAFDBEDD	{"k3":"v3"}	[103,200]	{"c1":10,"c2":"g"}	2013-03-03	20140330
+103	1	4	1.3	0.9	jkl	2014-04-04 04:04:04.444444444	abcd 	abcd	68692CCAC0BDE7	{"k4":"v4"}	[104,200]	{"c1":10,"c2":"j"}	2014-04-04	20140330
+104	2	5	1.4	1.2	mno	2015-05-05 05:05:05.555555555	abcde	abcde	B4F3CAFDBEDD	{"k5":"v5"}	[105,200]	{"c1":10,"c2":"m"}	2015-05-05	20140330
+105	3	1	1.0	1.5	pqr	2016-06-06 06:06:06.666666666	abcde	abcdef	68692CCAC0BDE7	{"k6":"v6"}	[106,200]	{"c1":10,"c2":"p"}	2016-06-06	20140330
+106	1	2	1.1	1.8	stu	2017-07-07 07:07:07.777777777	abcde	abcdefg	B4F3CAFDBEDD	{"k7":"v7"}	[107,200]	{"c1":10,"c2":"s"}	2017-07-07	20140330
+107	2	3	1.2	2.1	vwx	2018-08-08 08:08:08.888888888	bcdef	abcdefgh	68692CCAC0BDE7	{"k8":"v8"}	[108,200]	{"c1":10,"c2":"v"}	2018-08-08	20140330
+108	3	4	1.3	2.4	yza	2019-09-09 09:09:09.999999999	cdefg	B4F3CAFDBE	68656C6C6F	{"k9":"v9"}	[109,200]	{"c1":10,"c2":"y"}	2019-09-09	20140330
+109	1	5	1.4	2.7	bcd	2020-10-10 10:10:10.101010101	klmno	abcdedef	68692CCAC0BDE7	{"k10":"v10"}	[110,200]	{"c1":10,"c2":"b"}	2020-10-10	20140330
+110	2	1	1.0	3.0	efg	2021-11-11 11:11:11.111111111	pqrst	abcdede	B4F3CAFDBEDD	{"k11":"v11"}	[111,200]	{"c1":10,"c2":"e"}	2021-11-11	20140330
+111	3	2	1.1	3.3	hij	2022-12-12 12:12:12.121212121	nopqr	abcded	68692CCAC0BDE7	{"k12":"v12"}	[112,200]	{"c1":10,"c2":"h"}	2022-12-12	20140330
+112	1	3	1.2	3.6	klm	2023-01-02 13:13:13.131313131	opqrs	abcdd	B4F3CAFDBEDD	{"k13":"v13"}	[113,200]	{"c1":10,"c2":"k"}	2023-01-02	20140330
+113	2	4	1.3	3.9	nop	2024-02-02 14:14:14.141414141	pqrst	abc	68692CCAC0BDE7	{"k14":"v14"}	[114,200]	{"c1":10,"c2":"n"}	2024-02-02	20140330
+114	3	5	1.4	4.2	qrs	2025-03-03 15:15:15.151515151	qrstu	b	B4F3CAFDBEDD	{"k15":"v15"}	[115,200]	{"c1":10,"c2":"q"}	2025-03-03	20140330
+115	1	1	1.0	4.5	qrs	2026-04-04 16:16:16.161616161	rstuv	abcded	68692CCAC0BDE7	{"k16":"v16"}	[116,200]	{"c1":10,"c2":"q"}	2026-04-04	20140330
+116	2	2	1.1	4.8	wxy	2027-05-05 17:17:17.171717171	stuvw	abcded	B4F3CAFDBEDD	{"k17":"v17"}	[117,200]	{"c1":10,"c2":"w"}	2027-05-05	20140330
+117	3	3	1.2	5.1	zab	2028-06-06 18:18:18.181818181	tuvwx	abcded	68692CCAC0BDE7	{"k18":"v18"}	[118,200]	{"c1":10,"c2":"z"}	2028-06-06	20140330
+118	1	4	1.3	5.4	cde	2029-07-07 19:19:19.191919191	uvwzy	abcdede	B4F3CAFDBEDD	{"k19":"v19"}	[119,200]	{"c1":10,"c2":"c"}	2029-07-07	20140330
+119	2	5	1.4	5.7	fgh	2030-08-08 20:20:20.202020202	vwxyz	abcdede	68692CCAC0BDE7	{"k20":"v20"}	[120,200]	{"c1":10,"c2":"f"}	2030-08-08	20140330
+120	3	1	1.0	6.0	ijk	2031-09-09 21:21:21.212121212	wxyza	abcde	B4F3CAFDBEDD	{"k21":"v21"}	[121,200]	{"c1":10,"c2":"i"}	2031-09-09	20140330
+121	1	2	1.1	6.3	lmn	2032-10-10 22:22:22.222222222	bcdef	abcde		{"k22":"v22"}	[122,200]	{"c1":10,"c2":"l"}	2032-10-10	20140330
+PREHOOK: query: DESCRIBE FORMATTED parquet_mixed_partition_formats PARTITION (dateint=20140330)
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@parquet_mixed_partition_formats
+POSTHOOK: query: DESCRIBE FORMATTED parquet_mixed_partition_formats PARTITION (dateint=20140330)
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@parquet_mixed_partition_formats
+# col_name            	data_type           	comment             
+	 	 
+cint                	int                 	                    
+ctinyint            	tinyint             	                    
+csmallint           	smallint            	                    
+cfloat              	float               	                    
+cdouble             	double              	                    
+cstring1            	string              	                    
+t                   	timestamp           	                    
+cchar               	char(5)             	                    
+cvarchar            	varchar(10)         	                    
+cbinary             	string              	                    
+m1                  	map<string,varchar(3)>	                    
+l1                  	array<int>          	                    
+st1                 	struct<c1:int,c2:char(1)>	                    
+d                   	date                	                    
+	 	 
+# Partition Information	 	 
+# col_name            	data_type           	comment             
+	 	 
+dateint             	int                 	                    
+	 	 
+# Detailed Partition Information	 	 
+Partition Value:    	[20140330]          	 
+Database:           	default             	 
+Table:              	parquet_mixed_partition_formats	 
+#### A masked pattern was here ####
+Partition Parameters:	 	 
+	COLUMN_STATS_ACCURATE	true                
+	numFiles            	1                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	2521                
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	colelction.delim    	,                   
+	field.delim         	|                   
+	mapkey.delim        	:                   
+	serialization.format	|                   
+PREHOOK: query: ---change table serde and file format to PARQUET----
+
+ALTER TABLE parquet_mixed_partition_formats
+     SET FILEFORMAT
+     INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+     OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+     SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+PREHOOK: type: ALTERTABLE_FILEFORMAT
+PREHOOK: Input: default@parquet_mixed_partition_formats
+PREHOOK: Output: default@parquet_mixed_partition_formats
+POSTHOOK: query: ---change table serde and file format to PARQUET----
+
+ALTER TABLE parquet_mixed_partition_formats
+     SET FILEFORMAT
+     INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+     OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+     SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+POSTHOOK: type: ALTERTABLE_FILEFORMAT
+POSTHOOK: Input: default@parquet_mixed_partition_formats
+POSTHOOK: Output: default@parquet_mixed_partition_formats
+PREHOOK: query: DESCRIBE FORMATTED parquet_mixed_partition_formats
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@parquet_mixed_partition_formats
+POSTHOOK: query: DESCRIBE FORMATTED parquet_mixed_partition_formats
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@parquet_mixed_partition_formats
+# col_name            	data_type           	comment             
+	 	 
+cint                	int                 	                    
+ctinyint            	tinyint             	                    
+csmallint           	smallint            	                    
+cfloat              	float               	                    
+cdouble             	double              	                    
+cstring1            	string              	                    
+t                   	timestamp           	                    
+cchar               	char(5)             	                    
+cvarchar            	varchar(10)         	                    
+cbinary             	string              	                    
+m1                  	map<string,varchar(3)>	                    
+l1                  	array<int>          	                    
+st1                 	struct<c1:int,c2:char(1)>	                    
+d                   	date                	                    
+	 	 
+# Partition Information	 	 
+# col_name            	data_type           	comment             
+	 	 
+dateint             	int                 	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe	 
+InputFormat:        	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	colelction.delim    	,                   
+	field.delim         	|                   
+	mapkey.delim        	:                   
+	serialization.format	|                   
+PREHOOK: query: DESCRIBE FORMATTED parquet_mixed_partition_formats PARTITION (dateint=20140330)
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@parquet_mixed_partition_formats
+POSTHOOK: query: DESCRIBE FORMATTED parquet_mixed_partition_formats PARTITION (dateint=20140330)
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@parquet_mixed_partition_formats
+# col_name            	data_type           	comment             
+	 	 
+cint                	int                 	                    
+ctinyint            	tinyint             	                    
+csmallint           	smallint            	                    
+cfloat              	float               	                    
+cdouble             	double              	                    
+cstring1            	string              	                    
+t                   	timestamp           	                    
+cchar               	char(5)             	                    
+cvarchar            	varchar(10)         	                    
+cbinary             	string              	                    
+m1                  	map<string,varchar(3)>	                    
+l1                  	array<int>          	                    
+st1                 	struct<c1:int,c2:char(1)>	                    
+d                   	date                	                    
+	 	 
+# Partition Information	 	 
+# col_name            	data_type           	comment             
+	 	 
+dateint             	int                 	                    
+	 	 
+# Detailed Partition Information	 	 
+Partition Value:    	[20140330]          	 
+Database:           	default             	 
+Table:              	parquet_mixed_partition_formats	 
+#### A masked pattern was here ####
+Partition Parameters:	 	 
+	COLUMN_STATS_ACCURATE	true                
+	numFiles            	1                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	2521                
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	colelction.delim    	,                   
+	field.delim         	|                   
+	mapkey.delim        	:                   
+	serialization.format	|                   
+PREHOOK: query: SELECT * FROM parquet_mixed_partition_formats
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_mixed_partition_formats
+PREHOOK: Input: default@parquet_mixed_partition_formats@dateint=20140330
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM parquet_mixed_partition_formats
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_mixed_partition_formats
+POSTHOOK: Input: default@parquet_mixed_partition_formats@dateint=20140330
+#### A masked pattern was here ####
+100	1	1	1.0	0.0	abc	2011-01-01 01:01:01.111111111	a    	a  	B4F3CAFDBEDD	{"k1":"v1"}	[101,200]	{"c1":10,"c2":"a"}	2011-01-01	20140330
+101	2	2	1.1	0.3	def	2012-02-02 02:02:02.222222222	ab   	ab 	68692CCAC0BDE7	{"k2":"v2"}	[102,200]	{"c1":10,"c2":"d"}	2012-02-02	20140330
+102	3	3	1.2	0.6	ghi	2013-03-03 03:03:03.333333333	abc  	abc	B4F3CAFDBEDD	{"k3":"v3"}	[103,200]	{"c1":10,"c2":"g"}	2013-03-03	20140330
+103	1	4	1.3	0.9	jkl	2014-04-04 04:04:04.444444444	abcd 	abcd	68692CCAC0BDE7	{"k4":"v4"}	[104,200]	{"c1":10,"c2":"j"}	2014-04-04	20140330
+104	2	5	1.4	1.2	mno	2015-05-05 05:05:05.555555555	abcde	abcde	B4F3CAFDBEDD	{"k5":"v5"}	[105,200]	{"c1":10,"c2":"m"}	2015-05-05	20140330
+105	3	1	1.0	1.5	pqr	2016-06-06 06:06:06.666666666	abcde	abcdef	68692CCAC0BDE7	{"k6":"v6"}	[106,200]	{"c1":10,"c2":"p"}	2016-06-06	20140330
+106	1	2	1.1	1.8	stu	2017-07-07 07:07:07.777777777	abcde	abcdefg	B4F3CAFDBEDD	{"k7":"v7"}	[107,200]	{"c1":10,"c2":"s"}	2017-07-07	20140330
+107	2	3	1.2	2.1	vwx	2018-08-08 08:08:08.888888888	bcdef	abcdefgh	68692CCAC0BDE7	{"k8":"v8"}	[108,200]	{"c1":10,"c2":"v"}	2018-08-08	20140330
+108	3	4	1.3	2.4	yza	2019-09-09 09:09:09.999999999	cdefg	B4F3CAFDBE	68656C6C6F	{"k9":"v9"}	[109,200]	{"c1":10,"c2":"y"}	2019-09-09	20140330
+109	1	5	1.4	2.7	bcd	2020-10-10 10:10:10.101010101	klmno	abcdedef	68692CCAC0BDE7	{"k10":"v10"}	[110,200]	{"c1":10,"c2":"b"}	2020-10-10	20140330
+110	2	1	1.0	3.0	efg	2021-11-11 11:11:11.111111111	pqrst	abcdede	B4F3CAFDBEDD	{"k11":"v11"}	[111,200]	{"c1":10,"c2":"e"}	2021-11-11	20140330
+111	3	2	1.1	3.3	hij	2022-12-12 12:12:12.121212121	nopqr	abcded	68692CCAC0BDE7	{"k12":"v12"}	[112,200]	{"c1":10,"c2":"h"}	2022-12-12	20140330
+112	1	3	1.2	3.6	klm	2023-01-02 13:13:13.131313131	opqrs	abcdd	B4F3CAFDBEDD	{"k13":"v13"}	[113,200]	{"c1":10,"c2":"k"}	2023-01-02	20140330
+113	2	4	1.3	3.9	nop	2024-02-02 14:14:14.141414141	pqrst	abc	68692CCAC0BDE7	{"k14":"v14"}	[114,200]	{"c1":10,"c2":"n"}	2024-02-02	20140330
+114	3	5	1.4	4.2	qrs	2025-03-03 15:15:15.151515151	qrstu	b	B4F3CAFDBEDD	{"k15":"v15"}	[115,200]	{"c1":10,"c2":"q"}	2025-03-03	20140330
+115	1	1	1.0	4.5	qrs	2026-04-04 16:16:16.161616161	rstuv	abcded	68692CCAC0BDE7	{"k16":"v16"}	[116,200]	{"c1":10,"c2":"q"}	2026-04-04	20140330
+116	2	2	1.1	4.8	wxy	2027-05-05 17:17:17.171717171	stuvw	abcded	B4F3CAFDBEDD	{"k17":"v17"}	[117,200]	{"c1":10,"c2":"w"}	2027-05-05	20140330
+117	3	3	1.2	5.1	zab	2028-06-06 18:18:18.181818181	tuvwx	abcded	68692CCAC0BDE7	{"k18":"v18"}	[118,200]	{"c1":10,"c2":"z"}	2028-06-06	20140330
+118	1	4	1.3	5.4	cde	2029-07-07 19:19:19.191919191	uvwzy	abcdede	B4F3CAFDBEDD	{"k19":"v19"}	[119,200]	{"c1":10,"c2":"c"}	2029-07-07	20140330
+119	2	5	1.4	5.7	fgh	2030-08-08 20:20:20.202020202	vwxyz	abcdede	68692CCAC0BDE7	{"k20":"v20"}	[120,200]	{"c1":10,"c2":"f"}	2030-08-08	20140330
+120	3	1	1.0	6.0	ijk	2031-09-09 21:21:21.212121212	wxyza	abcde	B4F3CAFDBEDD	{"k21":"v21"}	[121,200]	{"c1":10,"c2":"i"}	2031-09-09	20140330
+121	1	2	1.1	6.3	lmn	2032-10-10 22:22:22.222222222	bcdef	abcde		{"k22":"v22"}	[122,200]	{"c1":10,"c2":"l"}	2032-10-10	20140330


[32/50] [abbrv] hive git commit: HIVE-11705 : refactor SARG stripe filtering for ORC into a separate method (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by xu...@apache.org.
HIVE-11705 : refactor SARG stripe filtering for ORC into a separate method (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ba0b33c1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ba0b33c1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ba0b33c1

Branch: refs/heads/spark
Commit: ba0b33c1025625b92bd669da60d2789f315e27f7
Parents: bc62a46
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Sep 15 18:09:54 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Sep 15 18:09:54 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   | 151 ++++++++++++-------
 .../apache/hadoop/hive/ql/io/orc/OrcSerde.java  |   1 +
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |   4 +-
 .../hive/ql/io/parquet/ProjectionPusher.java    |   3 +-
 .../hive/ql/io/sarg/ConvertAstToSearchArg.java  |   4 +
 .../ql/optimizer/ColumnPrunerProcFactory.java   |   3 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   2 +
 .../hive/serde2/ColumnProjectionUtils.java      |  22 +++
 .../hive/ql/io/sarg/SearchArgumentFactory.java  |   5 +-
 .../hive/ql/io/sarg/SearchArgumentImpl.java     |   7 +-
 10 files changed, 142 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ba0b33c1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index cf8694e..2500fb6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@ -58,11 +58,13 @@ import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
 import org.apache.hadoop.hive.ql.io.InputFormatChecker;
 import org.apache.hadoop.hive.ql.io.RecordIdentifier;
 import org.apache.hadoop.hive.ql.io.StatsProvidingRecordReader;
+import org.apache.hadoop.hive.ql.io.orc.OrcFile.WriterVersion;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.Context;
 import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.serde2.SerDeStats;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -265,8 +267,7 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
     OrcProto.Type root = types.get(rootColumn);
     for(int i=0; i < root.getSubtypesCount(); ++i) {
       if (included.contains(i)) {
-        includeColumnRecursive(types, result, root.getSubtypes(i),
-            rootColumn);
+        includeColumnRecursive(types, result, root.getSubtypes(i), rootColumn);
       }
     }
     return result;
@@ -292,6 +293,13 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
     int rootColumn = getRootColumn(isOriginal);
     String[] columnNames = new String[types.size() - rootColumn];
     int i = 0;
+    // The way this works is as such. originalColumnNames is the equivalent on getNeededColumns
+    // from TSOP. They are assumed to be in the same order as the columns in ORC file, AND they are
+    // assumed to be equivalent to the columns in includedColumns (because it was generated from
+    // the same column list at some point in the past), minus the subtype columns. Therefore, when
+    // we go thru all the top level ORC file columns that are included, in order, they match
+    // originalColumnNames. This way, we do not depend on names stored inside ORC for SARG leaf
+    // column name resolution (see mapSargColumns method).
     for(int columnId: types.get(rootColumn).getSubtypesList()) {
       if (includedColumns == null || includedColumns[columnId - rootColumn]) {
         // this is guaranteed to be positive because types only have children
@@ -306,8 +314,8 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
                                 List<OrcProto.Type> types,
                                 Configuration conf,
                                 boolean isOriginal) {
-    String columnNamesString = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR);
-    if (columnNamesString == null) {
+    String neededColumnNames = getNeededColumnNamesString(conf);
+    if (neededColumnNames == null) {
       LOG.debug("No ORC pushdown predicate - no column names");
       options.searchArgument(null, null);
       return;
@@ -321,9 +329,39 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
 
     LOG.info("ORC pushdown predicate: " + sarg);
     options.searchArgument(sarg, getSargColumnNames(
-        columnNamesString.split(","), types, options.getInclude(), isOriginal));
+        neededColumnNames.split(","), types, options.getInclude(), isOriginal));
   }
 
+  static boolean canCreateSargFromConf(Configuration conf) {
+    if (getNeededColumnNamesString(conf) == null) {
+      LOG.debug("No ORC pushdown predicate - no column names");
+      return false;
+    }
+    if (!ConvertAstToSearchArg.canCreateFromConf(conf)) {
+      LOG.debug("No ORC pushdown predicate");
+      return false;
+    }
+    return true;
+  }
+
+  private static String[] extractNeededColNames(
+      List<OrcProto.Type> types, Configuration conf, boolean[] include, boolean isOriginal) {
+    return extractNeededColNames(types, getNeededColumnNamesString(conf), include, isOriginal);
+  }
+
+  private static String[] extractNeededColNames(
+      List<OrcProto.Type> types, String columnNamesString, boolean[] include, boolean isOriginal) {
+    return getSargColumnNames(columnNamesString.split(","), types, include, isOriginal);
+  }
+
+  private static String getNeededColumnNamesString(Configuration conf) {
+    return conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR);
+  }
+
+  private static String getSargColumnIDsString(Configuration conf) {
+    return conf.getBoolean(ColumnProjectionUtils.READ_ALL_COLUMNS, true) ? null
+        : conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR);
+  }
   @Override
   public boolean validateInput(FileSystem fs, HiveConf conf,
                                ArrayList<FileStatus> files
@@ -863,34 +901,11 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
 
       // we can't eliminate stripes if there are deltas because the
       // deltas may change the rows making them match the predicate.
-      if (deltas.isEmpty()) {
-        Reader.Options options = new Reader.Options();
-        options.include(includedCols);
-        setSearchArgument(options, types, context.conf, isOriginal);
-        // only do split pruning if HIVE-8732 has been fixed in the writer
-        if (options.getSearchArgument() != null &&
-            writerVersion != OrcFile.WriterVersion.ORIGINAL) {
-          SearchArgument sarg = options.getSearchArgument();
-          List<PredicateLeaf> sargLeaves = sarg.getLeaves();
-          List<StripeStatistics> stripeStats = metadata.getStripeStatistics();
-          int[] filterColumns = RecordReaderImpl.mapSargColumns(sargLeaves,
-              options.getColumnNames(), getRootColumn(isOriginal));
-
-          if (stripeStats != null) {
-            // eliminate stripes that doesn't satisfy the predicate condition
-            includeStripe = new boolean[stripes.size()];
-            for (int i = 0; i < stripes.size(); ++i) {
-              includeStripe[i] = (i >= stripeStats.size()) ||
-                  isStripeSatisfyPredicate(stripeStats.get(i), sarg,
-                      filterColumns);
-              if (isDebugEnabled && !includeStripe[i]) {
-                LOG.debug("Eliminating ORC stripe-" + i + " of file '" +
-                    file.getPath() + "'  as it did not satisfy " +
-                    "predicate condition.");
-              }
-            }
-          }
-        }
+      if (deltas.isEmpty() && canCreateSargFromConf(context.conf)) {
+        SearchArgument sarg = ConvertAstToSearchArg.createFromConf(context.conf);
+        String[] sargColNames = extractNeededColNames(types, context.conf, includedCols, isOriginal);
+        includeStripe = pickStripes(sarg, sargColNames, writerVersion, isOriginal,
+            metadata.getStripeStatistics(), stripes.size(), file.getPath());
       }
 
       // if we didn't have predicate pushdown, read everything
@@ -990,28 +1005,6 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
       }
       return orcReader.getRawDataSizeFromColIndices(internalColIds);
     }
-
-    private boolean isStripeSatisfyPredicate(StripeStatistics stripeStatistics,
-                                             SearchArgument sarg,
-                                             int[] filterColumns) {
-      List<PredicateLeaf> predLeaves = sarg.getLeaves();
-      TruthValue[] truthValues = new TruthValue[predLeaves.size()];
-      for (int pred = 0; pred < truthValues.length; pred++) {
-        if (filterColumns[pred] != -1) {
-
-          // column statistics at index 0 contains only the number of rows
-          ColumnStatistics stats = stripeStatistics.getColumnStatistics()[filterColumns[pred]];
-          truthValues[pred] = RecordReaderImpl.evaluatePredicate(stats, predLeaves.get(pred), null);
-        } else {
-
-          // parition column case.
-          // partition filter will be evaluated by partition pruner so
-          // we will not evaluate partition filter here.
-          truthValues[pred] = TruthValue.YES_NO_NULL;
-        }
-      }
-      return sarg.evaluate(truthValues).isNeeded();
-    }
   }
 
   static List<OrcSplit> generateSplitsInfo(Configuration conf)
@@ -1353,6 +1346,54 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
         directory);
   }
 
+  private static boolean[] pickStripes(SearchArgument sarg, String[] sargColNames,
+      WriterVersion writerVersion, boolean isOriginal, List<StripeStatistics> stripeStats,
+      int stripeCount, Path filePath) {
+    LOG.info("ORC pushdown predicate: " + sarg);
+    if (sarg == null || stripeStats == null || writerVersion == OrcFile.WriterVersion.ORIGINAL) {
+      return null; // only do split pruning if HIVE-8732 has been fixed in the writer
+    }
+    // eliminate stripes that doesn't satisfy the predicate condition
+    List<PredicateLeaf> sargLeaves = sarg.getLeaves();
+    int[] filterColumns = RecordReaderImpl.mapSargColumnsToOrcInternalColIdx(sargLeaves,
+        sargColNames, getRootColumn(isOriginal));
+    return pickStripesInternal(sarg, filterColumns, stripeStats, stripeCount, filePath);
+  }
+
+  private static boolean[] pickStripesInternal(SearchArgument sarg, int[] filterColumns,
+      List<StripeStatistics> stripeStats, int stripeCount, Path filePath) {
+    boolean[] includeStripe = new boolean[stripeCount];
+    for (int i = 0; i < includeStripe.length; ++i) {
+      includeStripe[i] = (i >= stripeStats.size()) ||
+          isStripeSatisfyPredicate(stripeStats.get(i), sarg, filterColumns);
+      if (isDebugEnabled && !includeStripe[i]) {
+        LOG.debug("Eliminating ORC stripe-" + i + " of file '" + filePath
+            + "'  as it did not satisfy predicate condition.");
+      }
+    }
+    return includeStripe;
+  }
+
+  private static boolean isStripeSatisfyPredicate(
+      StripeStatistics stripeStatistics, SearchArgument sarg, int[] filterColumns) {
+    List<PredicateLeaf> predLeaves = sarg.getLeaves();
+    TruthValue[] truthValues = new TruthValue[predLeaves.size()];
+    for (int pred = 0; pred < truthValues.length; pred++) {
+      if (filterColumns[pred] != -1) {
+
+        // column statistics at index 0 contains only the number of rows
+        ColumnStatistics stats = stripeStatistics.getColumnStatistics()[filterColumns[pred]];
+        truthValues[pred] = RecordReaderImpl.evaluatePredicate(stats, predLeaves.get(pred), null);
+      } else {
+
+        // parition column case.
+        // partition filter will be evaluated by partition pruner so
+        // we will not evaluate partition filter here.
+        truthValues[pred] = TruthValue.YES_NO_NULL;
+      }
+    }
+    return sarg.evaluate(truthValues).isNeeded();
+  }
 
   @VisibleForTesting
   static SplitStrategy determineSplitStrategy(Context context, FileSystem fs, Path dir,

http://git-wip-us.apache.org/repos/asf/hive/blob/ba0b33c1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSerde.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSerde.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSerde.java
index 8beff4b..595f3b3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSerde.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSerde.java
@@ -108,6 +108,7 @@ public class OrcSerde implements SerDe, VectorizedSerde {
     ArrayList<TypeInfo> fieldTypes =
         TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty);
     StructTypeInfo rootType = new StructTypeInfo();
+    // The source column names for ORC serde that will be used in the schema.
     rootType.setAllStructFieldNames(columnNames);
     rootType.setAllStructFieldTypeInfos(fieldTypes);
     inspector = OrcStruct.createObjectInspector(rootType);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba0b33c1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
index fcb3746..ba304ba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
@@ -137,7 +137,7 @@ class RecordReaderImpl implements RecordReader {
    *                   result
    * @return an array mapping the sarg leaves to concrete column numbers
    */
-  public static int[] mapSargColumns(List<PredicateLeaf> sargLeaves,
+  public static int[] mapSargColumnsToOrcInternalColIdx(List<PredicateLeaf> sargLeaves,
                              String[] columnNames,
                              int rootColumn) {
     int[] result = new int[sargLeaves.size()];
@@ -693,7 +693,7 @@ class RecordReaderImpl implements RecordReader {
         List<OrcProto.Type> types, int includedCount) {
       this.sarg = sarg;
       sargLeaves = sarg.getLeaves();
-      filterColumns = mapSargColumns(sargLeaves, columnNames, 0);
+      filterColumns = mapSargColumnsToOrcInternalColIdx(sargLeaves, columnNames, 0);
       this.rowIndexStride = rowIndexStride;
       // included will not be null, row options will fill the array with trues if null
       sargColumns = new boolean[includedCount];

http://git-wip-us.apache.org/repos/asf/hive/blob/ba0b33c1/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ProjectionPusher.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ProjectionPusher.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ProjectionPusher.java
index 4480600..4848efd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ProjectionPusher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ProjectionPusher.java
@@ -65,6 +65,7 @@ public class ProjectionPusher {
     }
   }
 
+  @Deprecated  // Uses deprecated methods on ColumnProjectionUtils
   private void pushProjectionsAndFilters(final JobConf jobConf,
       final String splitPath, final String splitPathWithNoSchema) {
 
@@ -136,7 +137,7 @@ public class ProjectionPusher {
         filterExprSerialized);
   }
 
-
+  @Deprecated // Uses deprecated methods on ColumnProjectionUtils
   public JobConf pushProjectionsAndFilters(JobConf jobConf, Path path)
       throws IOException {
     updateMrWork(jobConf);  // TODO: refactor this in HIVE-6366

http://git-wip-us.apache.org/repos/asf/hive/blob/ba0b33c1/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
index e034650..690b8c9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
@@ -433,4 +433,8 @@ public class ConvertAstToSearchArg {
     return null;
   }
 
+  public static boolean canCreateFromConf(Configuration conf) {
+    return conf.get(TableScanDesc.FILTER_EXPR_CONF_STR) != null || conf.get(SARG_PUSHDOWN) != null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba0b33c1/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
index 2dc15f9..b104a7d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
@@ -488,6 +488,9 @@ public final class ColumnPrunerProcFactory {
     }
   }
 
+  /** Sets up needed columns for TSOP. Mainly, transfers column names from input
+   * RowSchema as well as the needed virtual columns, into TableScanDesc.
+   */
   public static void setupNeededColumns(TableScanOperator scanOp, RowSchema inputRS,
       List<String> cols) throws SemanticException {
     List<Integer> neededColumnIds = new ArrayList<Integer>();

http://git-wip-us.apache.org/repos/asf/hive/blob/ba0b33c1/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 16957b6..1076dfd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -9324,6 +9324,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     if (top == null) {
+      // Determine row schema for TSOP.
+      // Include column names from SerDe, the partition and virtual columns.
       rwsch = new RowResolver();
       try {
         StructObjectInspector rowObjectInspector = (StructObjectInspector) tab

http://git-wip-us.apache.org/repos/asf/hive/blob/ba0b33c1/serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java
index 10086c5..cbad3b2 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java
@@ -22,9 +22,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
 
@@ -33,6 +36,7 @@ import com.google.common.collect.Lists;
  *
  */
 public final class ColumnProjectionUtils {
+  public static final Log LOG = LogFactory.getLog(ColumnProjectionUtils.class);
 
   public static final String READ_COLUMN_IDS_CONF_STR = "hive.io.file.readcolumn.ids";
   public static final String READ_ALL_COLUMNS = "hive.io.file.read.all.columns";
@@ -54,6 +58,7 @@ public final class ColumnProjectionUtils {
    * and appendReadColumns
    */
   @Deprecated
+  @VisibleForTesting
   public static void setReadColumnIDs(Configuration conf, List<Integer> ids) {
     setReadColumnIDConf(conf, READ_COLUMN_IDS_CONF_STR_DEFAULT);
     appendReadColumns(conf, ids);
@@ -102,8 +107,21 @@ public final class ColumnProjectionUtils {
     conf.setBoolean(READ_ALL_COLUMNS, false);
   }
 
+  /**
+   * This method appends read column information to configuration to use for PPD. It is
+   * currently called with information from TSOP. Names come from TSOP input RowSchema, and
+   * IDs are the indexes inside the schema (which PPD assumes correspond to indexes inside the
+   * files to PPD in; something that would be invalid in many cases of schema evolution).
+   * @param conf Config to set values to.
+   * @param ids Column ids.
+   * @param names Column names.
+   */
   public static void appendReadColumns(
       Configuration conf, List<Integer> ids, List<String> names) {
+    if (ids.size() != names.size()) {
+      LOG.warn("Read column counts do not match: "
+          + ids.size() + " ids, " + names.size() + " names");
+    }
     appendReadColumns(conf, ids);
     appendReadColumnNames(conf, names);
   }
@@ -125,9 +143,13 @@ public final class ColumnProjectionUtils {
     List<Integer> result = new ArrayList<Integer>(list.length);
     for (String element : list) {
       // it may contain duplicates, remove duplicates
+      // TODO: WTF? This would break many assumptions elsewhere if it did.
+      //       Column names' and column ids' lists are supposed to be correlated.
       Integer toAdd = Integer.parseInt(element);
       if (!result.contains(toAdd)) {
         result.add(toAdd);
+      } else if (LOG.isInfoEnabled()) {
+        LOG.info("Duplicate ID " + toAdd + " in column ID list");
       }
     }
     return result;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba0b33c1/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
index 0778935..8fda95c 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
@@ -19,10 +19,13 @@
 package org.apache.hadoop.hive.ql.io.sarg;
 
 /**
- * A factory for creating SearchArguments.
+ * A factory for creating SearchArguments, as well as modifying those created by this factory.
  */
 public class SearchArgumentFactory {
   public static SearchArgument.Builder newBuilder() {
     return new SearchArgumentImpl.BuilderImpl();
   }
+  public static void setPredicateLeafColumn(PredicateLeaf leaf, String newName) {
+    SearchArgumentImpl.PredicateLeafImpl.setColumnName(leaf, newName);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba0b33c1/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
index d27ac16..a762b8b 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
@@ -40,7 +40,7 @@ final class SearchArgumentImpl implements SearchArgument {
   static final class PredicateLeafImpl implements PredicateLeaf {
     private final Operator operator;
     private final Type type;
-    private final String columnName;
+    private String columnName;
     private final Object literal;
     private final List<Object> literalList;
 
@@ -165,6 +165,11 @@ final class SearchArgumentImpl implements SearchArgument {
              (literalList == null ? 0 : literalList.hashCode()) *
                  103 * 101 * 3 * 17;
     }
+
+    public static void setColumnName(PredicateLeaf leaf, String newName) {
+      assert leaf instanceof PredicateLeafImpl;
+      ((PredicateLeafImpl)leaf).columnName = newName;
+    }
   }
 
 


[49/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1cce5f00
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1cce5f00
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1cce5f00

Branch: refs/heads/spark
Commit: 1cce5f006c595e67a4169851ceb352646759bc27
Parents: 201b1a0
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Wed Sep 16 09:41:25 2015 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Wed Sep 16 09:41:25 2015 -0700

----------------------------------------------------------------------
 .../rules/HiveAggregateProjectMergeRule.java    |  151 ++
 .../calcite/rules/HiveRelFieldTrimmer.java      |  145 +-
 .../translator/PlanModifierForASTConv.java      |    4 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    2 +
 .../alter_partition_coltype.q.out               |    8 +-
 .../clientpositive/annotate_stats_groupby.q.out |  106 +-
 .../annotate_stats_groupby2.q.out               |   28 +-
 .../results/clientpositive/auto_join18.q.out    |   12 +-
 .../auto_join18_multi_distinct.q.out            |   12 +-
 .../results/clientpositive/auto_join27.q.out    |   18 +-
 .../results/clientpositive/auto_join32.q.out    |    4 +-
 .../clientpositive/binarysortable_1.q.out       |  Bin 4329 -> 4325 bytes
 .../clientpositive/correlationoptimizer2.q.out  |  220 +-
 .../clientpositive/correlationoptimizer6.q.out  |  232 +-
 ql/src/test/results/clientpositive/count.q.out  |   14 +-
 .../results/clientpositive/ctas_colname.q.out   |   52 +-
 .../test/results/clientpositive/database.q.out  |    2 +-
 .../clientpositive/decimal_precision.q.out      |    4 +-
 .../results/clientpositive/decimal_udf.q.out    |   30 +-
 .../results/clientpositive/distinct_stats.q.out |   14 +-
 .../dynpart_sort_opt_vectorization.q.out        |  105 +-
 .../dynpart_sort_optimization.q.out             |  105 +-
 ...ryption_select_read_only_encrypted_tbl.q.out |    4 +-
 .../clientpositive/explain_logical.q.out        |   78 +-
 .../clientpositive/fetch_aggregation.q.out      |    4 +-
 .../test/results/clientpositive/gby_star.q.out  |   54 +-
 .../test/results/clientpositive/groupby12.q.out |    6 +-
 .../results/clientpositive/groupby5_map.q.out   |    4 +-
 .../clientpositive/groupby5_map_skew.q.out      |    4 +-
 .../results/clientpositive/groupby_cube1.q.out  |   12 +-
 .../groupby_distinct_samekey.q.out              |    6 +-
 .../clientpositive/groupby_grouping_sets2.q.out |   10 +-
 .../clientpositive/groupby_grouping_sets3.q.out |   12 +-
 .../clientpositive/groupby_grouping_sets5.q.out |    8 +-
 .../clientpositive/groupby_grouping_sets6.q.out |    8 +-
 .../clientpositive/groupby_position.q.out       |   36 +-
 .../clientpositive/groupby_resolution.q.out     |   60 +-
 .../clientpositive/groupby_rollup1.q.out        |   12 +-
 .../clientpositive/groupby_sort_10.q.out        |    8 +-
 .../clientpositive/groupby_sort_11.q.out        |   10 +-
 .../results/clientpositive/groupby_sort_8.q.out |   12 +-
 ql/src/test/results/clientpositive/having.q.out |   62 +-
 .../test/results/clientpositive/having2.q.out   |   12 +-
 .../clientpositive/index_auto_mult_tables.q.out |   12 +-
 .../clientpositive/index_auto_self_join.q.out   |   12 +-
 .../clientpositive/index_auto_update.q.out      |    6 +-
 .../index_bitmap_auto_partitioned.q.out         |    6 +-
 .../index_bitmap_compression.q.out              |    6 +-
 .../infer_bucket_sort_dyn_part.q.out            |    4 +-
 .../infer_bucket_sort_map_operators.q.out       |    4 +-
 ql/src/test/results/clientpositive/join18.q.out |   12 +-
 .../clientpositive/join18_multi_distinct.q.out  |   12 +-
 ql/src/test/results/clientpositive/join31.q.out |   36 +-
 .../limit_partition_metadataonly.q.out          |    4 +-
 .../results/clientpositive/limit_pushdown.q.out |   36 +-
 .../test/results/clientpositive/lineage2.q.out  |    2 +-
 .../test/results/clientpositive/lineage3.q.out  |    4 +-
 .../list_bucket_query_multiskew_3.q.out         |    2 +-
 .../clientpositive/mapjoin_mapjoin.q.out        |   32 +-
 .../clientpositive/metadata_only_queries.q.out  |    4 +-
 .../results/clientpositive/metadataonly1.q.out  |  112 +-
 .../results/clientpositive/multiMapJoin2.q.out  |  226 +-
 .../nonblock_op_deduplicate.q.out               |    8 +-
 .../results/clientpositive/nonmr_fetch.q.out    |   14 +-
 .../clientpositive/partition_multilevels.q.out  |    8 +-
 .../test/results/clientpositive/ppd_gby.q.out   |   12 +-
 .../test/results/clientpositive/ppd_gby2.q.out  |   60 +-
 .../clientpositive/ppd_join_filter.q.out        |   98 +-
 .../ql_rewrite_gbtoidx_cbo_1.q.out              |  168 +-
 .../ql_rewrite_gbtoidx_cbo_2.q.out              |   94 +-
 .../reduce_deduplicate_extended.q.out           |   32 +-
 .../clientpositive/selectDistinctStar.q.out     |   44 +-
 .../clientpositive/spark/auto_join18.q.out      |   10 +-
 .../spark/auto_join18_multi_distinct.q.out      |   12 +-
 .../clientpositive/spark/auto_join27.q.out      |   18 +-
 .../clientpositive/spark/auto_join32.q.out      |   53 +-
 .../results/clientpositive/spark/count.q.out    |   14 +-
 .../clientpositive/spark/groupby5_map.q.out     |    4 +-
 .../spark/groupby5_map_skew.q.out               |    4 +-
 .../clientpositive/spark/groupby_cube1.q.out    |   12 +-
 .../clientpositive/spark/groupby_position.q.out |   18 +-
 .../spark/groupby_resolution.q.out              |   60 +-
 .../clientpositive/spark/groupby_rollup1.q.out  |   12 +-
 .../results/clientpositive/spark/having.q.out   |   62 +-
 .../spark/infer_bucket_sort_map_operators.q.out |    4 +-
 .../results/clientpositive/spark/join18.q.out   |   10 +-
 .../spark/join18_multi_distinct.q.out           |   12 +-
 .../results/clientpositive/spark/join31.q.out   |   36 +-
 .../spark/limit_partition_metadataonly.q.out    |    4 +-
 .../clientpositive/spark/limit_pushdown.q.out   |   34 +-
 .../clientpositive/spark/mapjoin_mapjoin.q.out  |   24 +-
 .../spark/metadata_only_queries.q.out           |    4 +-
 .../clientpositive/spark/ppd_join_filter.q.out  |   90 +-
 .../spark/ql_rewrite_gbtoidx_cbo_1.q.out        |  168 +-
 .../clientpositive/spark/stats_only_null.q.out  |    8 +-
 .../clientpositive/spark/subquery_in.q.out      |   36 +-
 .../results/clientpositive/spark/union11.q.out  |   42 +-
 .../results/clientpositive/spark/union14.q.out  |   28 +-
 .../results/clientpositive/spark/union15.q.out  |   28 +-
 .../results/clientpositive/spark/union28.q.out  |    4 +-
 .../results/clientpositive/spark/union30.q.out  |    4 +-
 .../results/clientpositive/spark/union33.q.out  |    8 +-
 .../results/clientpositive/spark/union5.q.out   |   34 +-
 .../results/clientpositive/spark/union7.q.out   |   28 +-
 .../clientpositive/spark/union_remove_21.q.out  |    4 +-
 .../spark/vector_count_distinct.q.out           |    4 +-
 .../spark/vector_decimal_aggregate.q.out        |   12 +-
 .../spark/vector_distinct_2.q.out               |   28 +-
 .../clientpositive/spark/vector_groupby_3.q.out |   30 +-
 .../spark/vector_mapjoin_reduce.q.out           |   36 +-
 .../clientpositive/spark/vector_orderby_5.q.out |    6 +-
 .../clientpositive/spark/vectorization_0.q.out  |   16 +-
 .../clientpositive/spark/vectorization_13.q.out |   32 +-
 .../clientpositive/spark/vectorization_15.q.out |   16 +-
 .../clientpositive/spark/vectorization_16.q.out |   16 +-
 .../clientpositive/spark/vectorization_9.q.out  |   16 +-
 .../spark/vectorization_pushdown.q.out          |    4 +-
 .../spark/vectorization_short_regress.q.out     |   74 +-
 .../spark/vectorized_nested_mapjoin.q.out       |   18 +-
 .../spark/vectorized_timestamp_funcs.q.out      |   12 +-
 .../clientpositive/stats_only_null.q.out        |    8 +-
 .../results/clientpositive/stats_ppr_all.q.out  |   16 +-
 .../subq_where_serialization.q.out              |   18 +-
 .../clientpositive/subquery_exists_having.q.out |   48 +-
 .../results/clientpositive/subquery_in.q.out    |   36 +-
 .../clientpositive/subquery_in_having.q.out     |  260 +-
 .../clientpositive/subquery_notexists.q.out     |   18 +-
 .../subquery_notexists_having.q.out             |   26 +-
 .../results/clientpositive/subquery_notin.q.out |   24 +-
 .../subquery_notin_having.q.java1.7.out         |   50 +-
 .../subquery_unqualcolumnrefs.q.out             |   74 +-
 .../results/clientpositive/subquery_views.q.out |    8 +-
 .../test/results/clientpositive/tez/count.q.out |   14 +-
 .../tez/dynamic_partition_pruning.q.out         |   88 +-
 .../tez/dynpart_sort_opt_vectorization.q.out    |   90 +-
 .../tez/dynpart_sort_optimization.q.out         |   89 +-
 .../clientpositive/tez/explainuser_1.q.out      | 2319 +++++++++---------
 .../clientpositive/tez/explainuser_2.q.out      |  782 +++---
 .../results/clientpositive/tez/having.q.out     |   62 +-
 .../clientpositive/tez/limit_pushdown.q.out     |   34 +-
 .../clientpositive/tez/mapjoin_mapjoin.q.out    |   24 +-
 .../tez/metadata_only_queries.q.out             |    4 +-
 .../clientpositive/tez/metadataonly1.q.out      |   44 +-
 .../test/results/clientpositive/tez/mrr.q.out   |   94 +-
 .../clientpositive/tez/selectDistinctStar.q.out |   44 +-
 .../clientpositive/tez/stats_only_null.q.out    |    8 +-
 .../clientpositive/tez/subquery_in.q.out        |   36 +-
 .../results/clientpositive/tez/tez_dml.q.out    |    6 +-
 .../results/clientpositive/tez/union5.q.out     |   44 +-
 .../results/clientpositive/tez/union7.q.out     |   28 +-
 .../clientpositive/tez/unionDistinct_1.q.out    |    8 +-
 .../clientpositive/tez/vector_aggregate_9.q.out |    4 +-
 .../tez/vector_binary_join_groupby.q.out        |    4 +-
 .../tez/vector_count_distinct.q.out             |    4 +-
 .../tez/vector_decimal_aggregate.q.out          |   12 +-
 .../tez/vector_decimal_precision.q.out          |    4 +-
 .../clientpositive/tez/vector_decimal_udf.q.out |   30 +-
 .../clientpositive/tez/vector_distinct_2.q.out  |   28 +-
 .../clientpositive/tez/vector_groupby_3.q.out   |   30 +-
 .../tez/vector_groupby_reduce.q.out             |    8 +-
 .../tez/vector_grouping_sets.q.out              |    8 +-
 .../tez/vector_mapjoin_reduce.q.out             |   36 +-
 .../clientpositive/tez/vector_orderby_5.q.out   |    6 +-
 .../clientpositive/tez/vector_outer_join2.q.out |   20 +-
 .../tez/vector_partition_diff_num_cols.q.out    |   20 +-
 .../tez/vector_partitioned_date_time.q.out      |   12 +-
 .../tez/vector_reduce_groupby_decimal.q.out     |   24 +-
 .../clientpositive/tez/vectorization_0.q.out    |   16 +-
 .../clientpositive/tez/vectorization_13.q.out   |   32 +-
 .../clientpositive/tez/vectorization_15.q.out   |   16 +-
 .../clientpositive/tez/vectorization_16.q.out   |   16 +-
 .../clientpositive/tez/vectorization_9.q.out    |   16 +-
 .../tez/vectorization_limit.q.out               |   14 +-
 .../tez/vectorization_pushdown.q.out            |    4 +-
 .../tez/vectorization_short_regress.q.out       |   74 +-
 .../tez/vectorized_distinct_gby.q.out           |    8 +-
 .../vectorized_dynamic_partition_pruning.q.out  |   88 +-
 .../tez/vectorized_nested_mapjoin.q.out         |   18 +-
 .../clientpositive/tez/vectorized_parquet.q.out |    6 +-
 .../tez/vectorized_timestamp_funcs.q.out        |   12 +-
 ql/src/test/results/clientpositive/udf8.q.out   |    4 +-
 .../test/results/clientpositive/udf_count.q.out |   16 +-
 .../test/results/clientpositive/union11.q.out   |   70 +-
 .../test/results/clientpositive/union14.q.out   |   32 +-
 .../test/results/clientpositive/union15.q.out   |   38 +-
 .../test/results/clientpositive/union28.q.out   |    8 +-
 .../test/results/clientpositive/union30.q.out   |    8 +-
 .../test/results/clientpositive/union33.q.out   |    8 +-
 ql/src/test/results/clientpositive/union5.q.out |   48 +-
 ql/src/test/results/clientpositive/union7.q.out |   32 +-
 .../clientpositive/unionDistinct_1.q.out        |    8 +-
 .../clientpositive/union_remove_21.q.out        |    8 +-
 .../clientpositive/vector_aggregate_9.q.out     |    4 +-
 .../vector_aggregate_without_gby.q.out          |    4 +-
 .../vector_binary_join_groupby.q.out            |    4 +-
 .../clientpositive/vector_count_distinct.q.out  |    6 +-
 .../vector_decimal_aggregate.q.out              |   12 +-
 .../vector_decimal_precision.q.out              |    4 +-
 .../clientpositive/vector_decimal_udf.q.out     |   30 +-
 .../clientpositive/vector_distinct_2.q.out      |   28 +-
 .../clientpositive/vector_groupby_3.q.out       |   30 +-
 .../clientpositive/vector_groupby_reduce.q.out  |    8 +-
 .../clientpositive/vector_grouping_sets.q.out   |    8 +-
 .../clientpositive/vector_left_outer_join.q.out |    8 +-
 .../clientpositive/vector_mapjoin_reduce.q.out  |   36 +-
 .../clientpositive/vector_orderby_5.q.out       |    6 +-
 .../clientpositive/vector_outer_join1.q.out     |    8 +-
 .../clientpositive/vector_outer_join2.q.out     |   28 +-
 .../clientpositive/vector_outer_join3.q.out     |   24 +-
 .../clientpositive/vector_outer_join4.q.out     |    8 +-
 .../clientpositive/vector_outer_join5.q.out     |   48 +-
 .../vector_partition_diff_num_cols.q.out        |   20 +-
 .../vector_partitioned_date_time.q.out          |   12 +-
 .../vector_reduce_groupby_decimal.q.out         |   24 +-
 .../clientpositive/vectorization_0.q.out        |   16 +-
 .../clientpositive/vectorization_13.q.out       |   32 +-
 .../clientpositive/vectorization_15.q.out       |   16 +-
 .../clientpositive/vectorization_16.q.out       |   16 +-
 .../clientpositive/vectorization_9.q.out        |   16 +-
 .../clientpositive/vectorization_limit.q.out    |   16 +-
 .../clientpositive/vectorization_pushdown.q.out |    4 +-
 .../vectorization_short_regress.q.out           |   74 +-
 .../vectorized_distinct_gby.q.out               |   12 +-
 .../vectorized_nested_mapjoin.q.out             |   26 +-
 .../clientpositive/vectorized_parquet.q.out     |    6 +-
 .../vectorized_parquet_types.q.out              |    6 +-
 .../vectorized_timestamp_funcs.q.out            |   12 +-
 227 files changed, 4818 insertions(+), 5017 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
new file mode 100644
index 0000000..53f04ee
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
@@ -0,0 +1,151 @@
+/**
+ * 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.optimizer.calcite.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Planner rule that recognizes a {@link HiveAggregate}
+ * on top of a {@link HiveProject} and if possible
+ * aggregate through the project or removes the project.
+ *
+ * <p>This is only possible when the grouping expressions and arguments to
+ * the aggregate functions are field references (i.e. not expressions).
+ *
+ * <p>In some cases, this rule has the effect of trimming: the aggregate will
+ * use fewer columns than the project did.
+ */
+public class HiveAggregateProjectMergeRule extends RelOptRule {
+  public static final HiveAggregateProjectMergeRule INSTANCE =
+      new HiveAggregateProjectMergeRule();
+
+  /** Private constructor. */
+  private HiveAggregateProjectMergeRule() {
+    super(
+        operand(HiveAggregate.class,
+            operand(HiveProject.class, any())));
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final HiveAggregate aggregate = call.rel(0);
+    final HiveProject project = call.rel(1);
+    RelNode x = apply(aggregate, project);
+    if (x != null) {
+      call.transformTo(x);
+    }
+  }
+
+  public static RelNode apply(HiveAggregate aggregate,
+      HiveProject project) {
+    final List<Integer> newKeys = Lists.newArrayList();
+    final Map<Integer, Integer> map = new HashMap<>();
+    for (int key : aggregate.getGroupSet()) {
+      final RexNode rex = project.getProjects().get(key);
+      if (rex instanceof RexInputRef) {
+        final int newKey = ((RexInputRef) rex).getIndex();
+        newKeys.add(newKey);
+        map.put(key, newKey);
+      } else {
+        // Cannot handle "GROUP BY expression"
+        return null;
+      }
+    }
+
+    final ImmutableBitSet newGroupSet = aggregate.getGroupSet().permute(map);
+    ImmutableList<ImmutableBitSet> newGroupingSets = null;
+    if (aggregate.indicator) {
+      newGroupingSets =
+          ImmutableBitSet.ORDERING.immutableSortedCopy(
+              ImmutableBitSet.permute(aggregate.getGroupSets(), map));
+    }
+
+    final ImmutableList.Builder<AggregateCall> aggCalls =
+        ImmutableList.builder();
+    for (AggregateCall aggregateCall : aggregate.getAggCallList()) {
+      final ImmutableList.Builder<Integer> newArgs = ImmutableList.builder();
+      for (int arg : aggregateCall.getArgList()) {
+        final RexNode rex = project.getProjects().get(arg);
+        if (rex instanceof RexInputRef) {
+          newArgs.add(((RexInputRef) rex).getIndex());
+        } else {
+          // Cannot handle "AGG(expression)"
+          return null;
+        }
+      }
+      final int newFilterArg;
+      if (aggregateCall.filterArg >= 0) {
+        final RexNode rex = project.getProjects().get(aggregateCall.filterArg);
+        if (!(rex instanceof RexInputRef)) {
+          return null;
+        }
+        newFilterArg = ((RexInputRef) rex).getIndex();
+      } else {
+        newFilterArg = -1;
+      }
+      aggCalls.add(aggregateCall.copy(newArgs.build(), newFilterArg));
+    }
+
+    final Aggregate newAggregate =
+        aggregate.copy(aggregate.getTraitSet(), project.getInput(),
+            aggregate.indicator, newGroupSet, newGroupingSets,
+            aggCalls.build());
+
+    // Add a project if the group set is not in the same order or
+    // contains duplicates.
+    RelNode rel = newAggregate;
+    if (!newKeys.equals(newGroupSet.asList())) {
+      final List<Integer> posList = Lists.newArrayList();
+      for (int newKey : newKeys) {
+        posList.add(newGroupSet.indexOf(newKey));
+      }
+      if (aggregate.indicator) {
+        for (int newKey : newKeys) {
+          posList.add(aggregate.getGroupCount() + newGroupSet.indexOf(newKey));
+        }
+      }
+      for (int i = newAggregate.getGroupCount()
+                   + newAggregate.getIndicatorCount();
+           i < newAggregate.getRowType().getFieldCount(); i++) {
+        posList.add(i);
+      }
+      rel = RelOptUtil.createProject(HiveProject.DEFAULT_PROJECT_FACTORY,
+          rel, posList);
+    }
+
+    return rel;
+  }
+}
+
+// End AggregateProjectMergeRule.java

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
index 4144674..a12fa2a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
@@ -24,11 +24,10 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
@@ -37,20 +36,19 @@ import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.RelFieldTrimmer;
 import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.IntPair;
 import org.apache.calcite.util.mapping.Mapping;
 import org.apache.calcite.util.mapping.MappingType;
 import org.apache.calcite.util.mapping.Mappings;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
 
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 
 public class HiveRelFieldTrimmer extends RelFieldTrimmer {
 
-  public HiveRelFieldTrimmer(SqlValidator validator) {
-    super(validator);
-  }
+  private final RelFactories.AggregateFactory aggregateFactory;
 
   public HiveRelFieldTrimmer(SqlValidator validator,
       RelFactories.ProjectFactory projectFactory,
@@ -62,6 +60,7 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
       RelFactories.SetOpFactory setOpFactory) {
     super(validator, projectFactory, filterFactory, joinFactory,
             semiJoinFactory, sortFactory, aggregateFactory, setOpFactory);
+    this.aggregateFactory = aggregateFactory;
   }
 
   /**
@@ -156,27 +155,127 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
 
     return new TrimResult(newJoin, mapping);
   }
-
-  protected TrimResult trimChild(
-      RelNode rel,
-      RelNode input,
+  /**
+   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
+   * {@link org.apache.calcite.rel.logical.LogicalAggregate}.
+   */
+  @Override
+  public TrimResult trimFields(
+      Aggregate aggregate,
       ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
-    Util.discard(rel);
-    if (input.getClass().getName().endsWith("MedMdrClassExtentRel")) {
-      // MedMdrJoinRule cannot handle Join of Project of
-      // MedMdrClassExtentRel, only naked MedMdrClassExtentRel.
-      // So, disable trimming.
-      fieldsUsed = ImmutableBitSet.range(input.getRowType().getFieldCount());
+    // Fields:
+    //
+    // | sys fields | group fields | indicator fields | agg functions |
+    //
+    // Two kinds of trimming:
+    //
+    // 1. If agg rel has system fields but none of these are used, create an
+    // agg rel with no system fields.
+    //
+    // 2. If aggregate functions are not used, remove them.
+    //
+    // But group and indicator fields stay, even if they are not used.
+
+    final RelDataType rowType = aggregate.getRowType();
+
+    // Compute which input fields are used.
+    // 1. group fields are always used
+    final ImmutableBitSet.Builder inputFieldsUsed =
+        ImmutableBitSet.builder(aggregate.getGroupSet());
+    // 2. agg functions
+    for (AggregateCall aggCall : aggregate.getAggCallList()) {
+      for (int i : aggCall.getArgList()) {
+        inputFieldsUsed.set(i);
+      }
+      if (aggCall.filterArg >= 0) {
+        inputFieldsUsed.set(aggCall.filterArg);
+      }
+    }
+
+    // Create input with trimmed columns.
+    final RelNode input = aggregate.getInput();
+    final Set<RelDataTypeField> inputExtraFields = Collections.emptySet();
+    final TrimResult trimResult =
+        trimChild(aggregate, input, inputFieldsUsed.build(), inputExtraFields);
+    final RelNode newInput = trimResult.left;
+    final Mapping inputMapping = trimResult.right;
+
+    // We have to return group keys and (if present) indicators.
+    // So, pretend that the consumer asked for them.
+    final int groupCount = aggregate.getGroupSet().cardinality();
+    final int indicatorCount = aggregate.getIndicatorCount();
+    fieldsUsed =
+        fieldsUsed.union(ImmutableBitSet.range(groupCount + indicatorCount));
+
+    // If the input is unchanged, and we need to project all columns,
+    // there's nothing to do.
+    if (input == newInput
+        && fieldsUsed.equals(ImmutableBitSet.range(rowType.getFieldCount()))) {
+      return new TrimResult(
+          aggregate,
+          Mappings.createIdentity(rowType.getFieldCount()));
     }
-    final ImmutableList<RelCollation> collations =
-        RelMetadataQuery.collations(input);
-    for (RelCollation collation : collations) {
-      for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
-        fieldsUsed = fieldsUsed.set(fieldCollation.getFieldIndex());
+
+    // Which agg calls are used by our consumer?
+    int j = groupCount + indicatorCount;
+    int usedAggCallCount = 0;
+    for (int i = 0; i < aggregate.getAggCallList().size(); i++) {
+      if (fieldsUsed.get(j++)) {
+        ++usedAggCallCount;
       }
     }
-    return dispatchTrimFields(input, fieldsUsed, extraFields);
+
+    // Offset due to the number of system fields having changed.
+    Mapping mapping =
+        Mappings.create(
+            MappingType.INVERSE_SURJECTION,
+            rowType.getFieldCount(),
+            groupCount + indicatorCount + usedAggCallCount);
+
+    final ImmutableBitSet newGroupSet =
+        Mappings.apply(inputMapping, aggregate.getGroupSet());
+
+    final ImmutableList<ImmutableBitSet> newGroupSets =
+        ImmutableList.copyOf(
+            Iterables.transform(aggregate.getGroupSets(),
+                new Function<ImmutableBitSet, ImmutableBitSet>() {
+                  @Override
+                  public ImmutableBitSet apply(ImmutableBitSet input) {
+                    return Mappings.apply(inputMapping, input);
+                  }
+                }));
+
+    // Populate mapping of where to find the fields. System, group key and
+    // indicator fields first.
+    for (j = 0; j < groupCount + indicatorCount; j++) {
+      mapping.set(j, j);
+    }
+
+    // Now create new agg calls, and populate mapping for them.
+    final List<AggregateCall> newAggCallList = new ArrayList<>();
+    j = groupCount + indicatorCount;
+    for (AggregateCall aggCall : aggregate.getAggCallList()) {
+      if (fieldsUsed.get(j)) {
+        AggregateCall newAggCall =
+            aggCall.copy(Mappings.apply2(inputMapping, aggCall.getArgList()),
+                Mappings.apply(inputMapping, aggCall.filterArg));
+        if (newAggCall.equals(aggCall)) {
+          newAggCall = aggCall; // immutable -> canonize to save space
+        }
+        mapping.set(j, groupCount + indicatorCount + newAggCallList.size());
+        newAggCallList.add(newAggCall);
+      }
+      ++j;
+    }
+
+    RelNode newAggregate = aggregateFactory.createAggregate(newInput,
+        aggregate.indicator, newGroupSet, newGroupSets, newAggCallList);
+
+    assert newAggregate.getClass() == aggregate.getClass();
+
+    return new TrimResult(newAggregate, mapping);
   }
 
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
index 67f17c2..16a375c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
@@ -265,8 +265,8 @@ public class PlanModifierForASTConv {
 
     // TODO: Verify GB having is not a separate filter (if so we shouldn't
     // introduce derived table)
-    if (parent instanceof Filter || parent instanceof Join
-        || parent instanceof SetOp) {
+    if (parent instanceof Filter || parent instanceof Join || parent instanceof SetOp ||
+       (parent instanceof Aggregate && filterNode.getInputs().get(0) instanceof Aggregate)) {
       validParent = false;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index d5c747f..0a7ce3a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -134,6 +134,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveAggregateProjectMergeRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveExpandDistinctAggregatesRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterJoinRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterProjectTransposeRule;
@@ -883,6 +884,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       hepPgmBldr.addRuleInstance(ProjectRemoveRule.INSTANCE);
       hepPgmBldr.addRuleInstance(UnionMergeRule.INSTANCE);
       hepPgmBldr.addRuleInstance(new ProjectMergeRule(false, HiveProject.DEFAULT_PROJECT_FACTORY));
+      hepPgmBldr.addRuleInstance(HiveAggregateProjectMergeRule.INSTANCE);
 
       hepPgm = hepPgmBldr.build();
       HepPlanner hepPlanner = new HepPlanner(hepPgm);

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_partition_coltype.q.out b/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
index 9fc3c8d..a42b464 100644
--- a/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
+++ b/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
@@ -232,8 +232,8 @@ STAGE PLANS:
               name: default.alter_coltype
             name: default.alter_coltype
       Truncated Path -> Alias:
-        /alter_coltype/dt=100/ts=3.0 [$hdt$_0:alter_coltype]
-        /alter_coltype/dt=100/ts=6.30 [$hdt$_0:alter_coltype]
+        /alter_coltype/dt=100/ts=3.0 [alter_coltype]
+        /alter_coltype/dt=100/ts=6.30 [alter_coltype]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -411,7 +411,7 @@ STAGE PLANS:
               name: default.alter_coltype
             name: default.alter_coltype
       Truncated Path -> Alias:
-        /alter_coltype/dt=100/ts=6.30 [$hdt$_0:alter_coltype]
+        /alter_coltype/dt=100/ts=6.30 [alter_coltype]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -568,7 +568,7 @@ STAGE PLANS:
               name: default.alter_coltype
             name: default.alter_coltype
       Truncated Path -> Alias:
-        /alter_coltype/dt=100/ts=3.0 [$hdt$_0:alter_coltype]
+        /alter_coltype/dt=100/ts=3.0 [alter_coltype]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out b/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
index 82cc0da..1b9ec68 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
@@ -157,11 +157,11 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: PARTIAL
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: PARTIAL
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string), _col1 (type: int)
+                keys: state (type: string), locid (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 8 Data size: 752 Basic stats: COMPLETE Column stats: PARTIAL
@@ -178,22 +178,18 @@ STAGE PLANS:
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 7 Data size: 658 Basic stats: COMPLETE Column stats: PARTIAL
-          Select Operator
-            expressions: _col0 (type: string), _col2 (type: bigint), _col1 (type: int)
+          Group By Operator
+            aggregations: min(_col1)
+            keys: _col0 (type: string), _col2 (type: bigint)
+            mode: hash
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 7 Data size: 658 Basic stats: COMPLETE Column stats: PARTIAL
-            Group By Operator
-              aggregations: min(_col2)
-              keys: _col0 (type: string), _col1 (type: bigint)
-              mode: hash
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 7 Data size: 686 Basic stats: COMPLETE Column stats: PARTIAL
-              File Output Operator
-                compressed: false
-                table:
-                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            Statistics: Num rows: 7 Data size: 686 Basic stats: COMPLETE Column stats: PARTIAL
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-2
     Map Reduce
@@ -255,10 +251,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: year (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: year
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: int)
+                keys: year (type: int)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 8 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
@@ -308,10 +304,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int)
+                keys: state (type: string), locid (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
@@ -361,10 +357,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 32 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
@@ -415,10 +411,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 24 Data size: 4200 Basic stats: COMPLETE Column stats: COMPLETE
@@ -469,10 +465,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 8 Data size: 1400 Basic stats: COMPLETE Column stats: COMPLETE
@@ -523,10 +519,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 16 Data size: 2800 Basic stats: COMPLETE Column stats: COMPLETE
@@ -577,10 +573,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 24 Data size: 4200 Basic stats: COMPLETE Column stats: COMPLETE
@@ -631,10 +627,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 32 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
@@ -689,10 +685,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: year (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: year
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: int)
+                keys: year (type: int)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 4 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
@@ -742,10 +738,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 16 Data size: 2800 Basic stats: COMPLETE Column stats: COMPLETE
@@ -798,10 +794,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: PARTIAL
             Select Operator
               expressions: state (type: string), zip (type: bigint)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, zip
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: PARTIAL
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: bigint)
+                keys: state (type: string), zip (type: bigint)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 4 Data size: 344 Basic stats: COMPLETE Column stats: PARTIAL
@@ -851,10 +847,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 32 Data size: 3184 Basic stats: COMPLETE Column stats: NONE
@@ -905,10 +901,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 24 Data size: 2388 Basic stats: COMPLETE Column stats: NONE
@@ -959,10 +955,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
@@ -1013,10 +1009,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 16 Data size: 1592 Basic stats: COMPLETE Column stats: NONE
@@ -1067,10 +1063,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 24 Data size: 2388 Basic stats: COMPLETE Column stats: NONE
@@ -1121,10 +1117,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 32 Data size: 3184 Basic stats: COMPLETE Column stats: NONE
@@ -1175,10 +1171,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: year (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: year
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: int)
+                keys: year (type: int)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
@@ -1228,10 +1224,10 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: state (type: string), locid (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, locid
               Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
+                keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 32 Data size: 3184 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out b/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out
index 2cb1e84..be3fa1d 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out
@@ -91,10 +91,10 @@ STAGE PLANS:
             Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: state (type: string), country (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, country
               Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: state (type: string), country (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
@@ -144,10 +144,10 @@ STAGE PLANS:
             Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: state (type: string), country (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, country
               Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
+                keys: state (type: string), country (type: string), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 80 Data size: 800 Basic stats: COMPLETE Column stats: NONE
@@ -202,10 +202,10 @@ STAGE PLANS:
             Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), country (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, country
               Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: state (type: string), country (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 8 Data size: 1384 Basic stats: COMPLETE Column stats: COMPLETE
@@ -257,10 +257,10 @@ STAGE PLANS:
             Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: PARTIAL
             Select Operator
               expressions: state (type: string), votes (type: bigint)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, votes
               Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: PARTIAL
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: bigint)
+                keys: state (type: string), votes (type: bigint)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: PARTIAL
@@ -310,10 +310,10 @@ STAGE PLANS:
             Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), country (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, country
               Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
+                keys: state (type: string), country (type: string), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 32 Data size: 8256 Basic stats: COMPLETE Column stats: COMPLETE
@@ -364,10 +364,10 @@ STAGE PLANS:
             Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), country (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, country
               Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: state (type: string), country (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 20 Data size: 3460 Basic stats: COMPLETE Column stats: COMPLETE
@@ -417,10 +417,10 @@ STAGE PLANS:
             Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), country (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: state, country
               Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
+                keys: state (type: string), country (type: string), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 80 Data size: 20640 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/auto_join18.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_join18.q.out b/ql/src/test/results/clientpositive/auto_join18.q.out
index 6dc7a63..7fd7dd1 100644
--- a/ql/src/test/results/clientpositive/auto_join18.q.out
+++ b/ql/src/test/results/clientpositive/auto_join18.q.out
@@ -40,11 +40,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -140,11 +140,11 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT value)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/auto_join18_multi_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_join18_multi_distinct.q.out b/ql/src/test/results/clientpositive/auto_join18_multi_distinct.q.out
index cc17ad1..0a9dd76 100644
--- a/ql/src/test/results/clientpositive/auto_join18_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/auto_join18_multi_distinct.q.out
@@ -42,11 +42,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -142,11 +142,11 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1), count(DISTINCT _col0)
-                keys: _col0 (type: string), _col1 (type: string)
+                aggregations: count(DISTINCT value), count(DISTINCT key)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/auto_join27.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_join27.q.out b/ql/src/test/results/clientpositive/auto_join27.q.out
index 16a7f02..9c03c78 100644
--- a/ql/src/test/results/clientpositive/auto_join27.q.out
+++ b/ql/src/test/results/clientpositive/auto_join27.q.out
@@ -39,20 +39,16 @@ STAGE PLANS:
             Filter Operator
               predicate: (UDFToDouble(key) < 200.0) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                keys: key (type: string), value (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/auto_join32.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_join32.q.out b/ql/src/test/results/clientpositive/auto_join32.q.out
index f862870..161ab6b 100644
--- a/ql/src/test/results/clientpositive/auto_join32.q.out
+++ b/ql/src/test/results/clientpositive/auto_join32.q.out
@@ -411,10 +411,10 @@ STAGE PLANS:
                   outputColumnNames: _col1, _col3
                   Select Operator
                     expressions: _col3 (type: string), _col1 (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: _col3, _col1
                     Group By Operator
                       aggregations: count(DISTINCT _col1)
-                      keys: _col0 (type: string), _col1 (type: string)
+                      keys: _col3 (type: string), _col1 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
                       Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/binarysortable_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/binarysortable_1.q.out b/ql/src/test/results/clientpositive/binarysortable_1.q.out
index 421fd2e..9ef9221 100644
Binary files a/ql/src/test/results/clientpositive/binarysortable_1.q.out and b/ql/src/test/results/clientpositive/binarysortable_1.q.out differ

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/correlationoptimizer2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/correlationoptimizer2.q.out b/ql/src/test/results/clientpositive/correlationoptimizer2.q.out
index c1a20c8..96c7660 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer2.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer2.q.out
@@ -41,22 +41,18 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                aggregations: count(value)
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -144,22 +140,18 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                aggregations: count(value)
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -228,44 +220,36 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                aggregations: count(value)
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
           TableScan
             alias: y
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                aggregations: count(value)
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Demux Operator
           Statistics: Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
@@ -411,11 +395,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -511,11 +495,11 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -592,11 +576,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -611,11 +595,11 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -770,11 +754,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -870,11 +854,11 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -951,11 +935,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -970,11 +954,11 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -1129,11 +1113,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1229,11 +1213,11 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -1310,11 +1294,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1329,11 +1313,11 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -1489,10 +1473,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1614,10 +1598,10 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -1703,10 +1687,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1720,10 +1704,10 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -2029,22 +2013,18 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                aggregations: count(value)
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -2144,22 +2124,18 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                aggregations: count(value)
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count(_col1)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Demux Operator
           Statistics: Num rows: 276 Data size: 2854 Basic stats: COMPLETE Column stats: NONE


[07/50] [abbrv] hive git commit: HIVE-11510 : Metatool updateLocation warning on views (Wei Zheng via Sushanth Sowmyan)

Posted by xu...@apache.org.
HIVE-11510 : Metatool updateLocation warning on views (Wei Zheng via Sushanth Sowmyan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/27bf8f0f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/27bf8f0f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/27bf8f0f

Branch: refs/heads/spark
Commit: 27bf8f0f70af198ce2c5d939046ca61ab7414585
Parents: b4be31f
Author: Sushanth Sowmyan <kh...@gmail.com>
Authored: Thu Sep 10 12:01:38 2015 -0700
Committer: Sushanth Sowmyan <kh...@gmail.com>
Committed: Thu Sep 10 12:14:57 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/ObjectStore.java       | 19 +++++++++++++++++--
 .../hive/metastore/tools/HiveMetaTool.java       |  5 +++++
 2 files changed, 22 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/27bf8f0f/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index d165fc8..4d6bfcc 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -5930,11 +5930,13 @@ public class ObjectStore implements RawStore, Configurable {
   public class UpdateMStorageDescriptorTblURIRetVal {
     private List<String> badRecords;
     private Map<String, String> updateLocations;
+    private int numNullRecords;
 
     UpdateMStorageDescriptorTblURIRetVal(List<String> badRecords,
-      Map<String, String> updateLocations) {
+      Map<String, String> updateLocations, int numNullRecords) {
       this.badRecords = badRecords;
       this.updateLocations = updateLocations;
+      this.numNullRecords = numNullRecords;
     }
 
     public List<String> getBadRecords() {
@@ -5952,6 +5954,14 @@ public class ObjectStore implements RawStore, Configurable {
     public void setUpdateLocations(Map<String, String> updateLocations) {
       this.updateLocations = updateLocations;
     }
+
+    public int getNumNullRecords() {
+      return numNullRecords;
+    }
+
+    public void setNumNullRecords(int numNullRecords) {
+      this.numNullRecords = numNullRecords;
+    }
   }
 
   /** The following APIs
@@ -5967,6 +5977,7 @@ public class ObjectStore implements RawStore, Configurable {
     Query query = null;
     Map<String, String> updateLocations = new HashMap<String, String>();
     List<String> badRecords = new ArrayList<String>();
+    int numNullRecords = 0;
     UpdateMStorageDescriptorTblURIRetVal retVal = null;
     try {
       openTransaction();
@@ -5976,6 +5987,10 @@ public class ObjectStore implements RawStore, Configurable {
       for (MStorageDescriptor mSDS : mSDSs) {
         URI locationURI = null;
         String location = mSDS.getLocation();
+        if (location == null) { // This can happen for View or Index
+          numNullRecords++;
+          continue;
+        }
         try {
           locationURI = new Path(location).toUri();
         } catch (IllegalArgumentException e) {
@@ -5995,7 +6010,7 @@ public class ObjectStore implements RawStore, Configurable {
       }
       committed = commitTransaction();
       if (committed) {
-        retVal = new UpdateMStorageDescriptorTblURIRetVal(badRecords, updateLocations);
+        retVal = new UpdateMStorageDescriptorTblURIRetVal(badRecords, updateLocations, numNullRecords);
       }
       return retVal;
     } finally {

http://git-wip-us.apache.org/repos/asf/hive/blob/27bf8f0f/metastore/src/java/org/apache/hadoop/hive/metastore/tools/HiveMetaTool.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/tools/HiveMetaTool.java b/metastore/src/java/org/apache/hadoop/hive/metastore/tools/HiveMetaTool.java
index 411ac21..e4e9e3a 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/tools/HiveMetaTool.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/tools/HiveMetaTool.java
@@ -222,6 +222,11 @@ public class HiveMetaTool {
           System.err.println("bad location URI: " + badRecord);
         }
       }
+      int numNullRecords = retVal.getNumNullRecords();
+      if (numNullRecords != 0) {
+        LOG.debug("Number of NULL location URI: " + numNullRecords +
+            ". This can happen for View or Index.");
+      }
     }
   }
 


[20/50] [abbrv] hive git commit: HIVE-11780: Add "set role none" support(Sun Dapeng, reviewed by Ferdinand Xu)

Posted by xu...@apache.org.
HIVE-11780: Add "set role none" support(Sun Dapeng, reviewed by Ferdinand Xu)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8bcd07d1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8bcd07d1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8bcd07d1

Branch: refs/heads/spark
Commit: 8bcd07d18d7791745b54b1775bbf54e26c23aeea
Parents: 66fb960
Author: Sun Dapeng <da...@intel.com>
Authored: Mon Sep 14 02:54:52 2015 -0400
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Mon Sep 14 02:54:52 2015 -0400

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g     | 2 ++
 .../plugin/sqlstd/SQLStdHiveAccessController.java            | 5 +++++
 .../clientpositive/authorization_set_show_current_role.q     | 3 +++
 .../clientpositive/authorization_set_show_current_role.q.out | 8 ++++++++
 4 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8bcd07d1/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index cf7ab3a..3969a54 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -1495,6 +1495,8 @@ setRole
     (
     (KW_ALL) => (all=KW_ALL) -> ^(TOK_SHOW_SET_ROLE Identifier[$all.text])
     |
+    (KW_NONE) => (none=KW_NONE) -> ^(TOK_SHOW_SET_ROLE Identifier[$none.text])
+    |
     identifier -> ^(TOK_SHOW_SET_ROLE identifier)
     )
     ;

http://git-wip-us.apache.org/repos/asf/hive/blob/8bcd07d1/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
index 9c78876..2f6e26b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
@@ -520,6 +520,11 @@ public class SQLStdHiveAccessController implements HiveAccessController {
     HiveAuthzPluginException {
 
     initUserRoles();
+    if (NONE.equalsIgnoreCase(roleName)) {
+      // for set role NONE, clear all roles for current session.
+      currentRoles.clear();
+      return;
+    }
     if (ALL.equalsIgnoreCase(roleName)) {
       // for set role ALL, reset roles to default roles.
       currentRoles.clear();

http://git-wip-us.apache.org/repos/asf/hive/blob/8bcd07d1/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q b/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q
index f10b649..50a5862 100644
--- a/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q
+++ b/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q
@@ -13,6 +13,9 @@ show current roles;
 set role PUBLIC;
 show current roles;
 
+set role NONE;
+show current roles;
+
 set role ALL;
 show current roles;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/8bcd07d1/ql/src/test/results/clientpositive/authorization_set_show_current_role.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_set_show_current_role.q.out b/ql/src/test/results/clientpositive/authorization_set_show_current_role.q.out
index 4ac4320..a2b273d 100644
--- a/ql/src/test/results/clientpositive/authorization_set_show_current_role.q.out
+++ b/ql/src/test/results/clientpositive/authorization_set_show_current_role.q.out
@@ -33,6 +33,14 @@ PREHOOK: type: SHOW_ROLES
 POSTHOOK: query: show current roles
 POSTHOOK: type: SHOW_ROLES
 public
+PREHOOK: query: set role NONE
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: set role NONE
+POSTHOOK: type: SHOW_ROLES
+PREHOOK: query: show current roles
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: show current roles
+POSTHOOK: type: SHOW_ROLES
 PREHOOK: query: set role ALL
 PREHOOK: type: SHOW_ROLES
 POSTHOOK: query: set role ALL


[21/50] [abbrv] hive git commit: HIVE-11792: User explain in tez does not preserve ordering (Prasanth Jayachandran reviewed by Pengcheng Xiong)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
index 9358158..79c7116 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
@@ -160,7 +160,7 @@ Stage-2
          File Output Operator [FS_6]
             compressed:false
             Statistics:Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_4]
             |  aggregations:["compute_stats(VALUE._col0)","compute_stats(VALUE._col1)"]
             |  outputColumnNames:["_col0","_col1"]
@@ -244,7 +244,7 @@ Stage-3
                      File Output Operator [FS_2]
                         compressed:false
                         Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","name:":"default.src_autho_test","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+                        table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","name:":"default.src_autho_test"}
                         Select Operator [SEL_1]
                            outputColumnNames:["_col0","_col1"]
                            Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -433,7 +433,7 @@ Stage-0
          File Output Operator [FS_5]
             compressed:false
             Statistics:Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Limit [LIM_4]
                Number of rows:5
                Statistics:Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
@@ -479,7 +479,7 @@ Stage-3
    Stats-Aggr Operator
       Stage-0
          Move Operator
-            table:{"serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.orc_merge5","input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat","serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.orc_merge5"}
             Stage-2
                Dependency Collection{}
                   Stage-5(CONDITIONAL)
@@ -491,7 +491,7 @@ Stage-3
                                  File Output Operator [FS_3]
                                     compressed:false
                                     Statistics:Num rows: 306 Data size: 82044 Basic stats: COMPLETE Column stats: NONE
-                                    table:{"serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.orc_merge5","input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"}
+                                    table:{"input format:":"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat","output format:":"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat","serde:":"org.apache.hadoop.hive.ql.io.orc.OrcSerde","name:":"default.orc_merge5"}
                                     Select Operator [SEL_2]
                                        outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
                                        Statistics:Num rows: 306 Data size: 82044 Basic stats: COMPLETE Column stats: NONE


[04/50] [abbrv] hive git commit: HIVE-11754 : Not reachable code parts in StatsUtils (Navis via Ashutosh Chauhan)

Posted by xu...@apache.org.
HIVE-11754 : Not reachable code parts in StatsUtils (Navis via Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7a71e50d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7a71e50d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7a71e50d

Branch: refs/heads/spark
Commit: 7a71e50d456070272d802eedb4a8468a4a1ab4af
Parents: 9b11caf
Author: Navis Ryu <na...@apache.org>
Authored: Tue Sep 8 20:22:00 2015 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Wed Sep 9 23:48:39 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/stats/StatsUtils.java | 84 ++++++--------------
 1 file changed, 25 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7a71e50d/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index 55aea0e..e1f38a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -66,32 +66,30 @@ import org.apache.hadoop.hive.serde2.objectinspector.StandardMapObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableBinaryObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableBooleanObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableByteObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantBinaryObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveCharObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveVarcharObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantStringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableDateObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableDoubleObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableFloatObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveCharObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveDecimalObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveVarcharObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableIntObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableLongObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableShortObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableStringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -569,7 +567,7 @@ public class StatsUtils {
    *          - hive conf
    * @param parts
    *          - partition list
-   * @return sizes of patitions
+   * @return sizes of partitions
    */
   public static List<Long> getFileSizeForPartitions(HiveConf conf, List<Partition> parts) {
     List<Long> sizes = Lists.newArrayList();
@@ -783,19 +781,9 @@ public class StatsUtils {
         ConstantObjectInspector coi = (ConstantObjectInspector) oi;
 
         // if writable constant is null then return size 0
-        if (coi.getWritableConstantValue() == null) {
-          return 0;
-        }
-
-        return coi.getWritableConstantValue().toString().length();
-      } else if (oi instanceof WritableConstantStringObjectInspector) {
-
-        // some UDFs return writable constant strings (fixed width)
-        // Ex: select upper("hello") from table
-        WritableConstantStringObjectInspector wcsoi = (WritableConstantStringObjectInspector) oi;
-
-        return wcsoi.getWritableConstantValue().toString().length();
-      } else if (oi instanceof WritableStringObjectInspector) {
+        Object constantValue = coi.getWritableConstantValue();
+        return constantValue == null ? 0 : constantValue.toString().length();
+      } else if (oi instanceof StringObjectInspector) {
 
         // some UDFs may emit strings of variable length. like pattern matching
         // UDFs. it's hard to find the length of such UDFs.
@@ -809,18 +797,11 @@ public class StatsUtils {
         ConstantObjectInspector coi = (ConstantObjectInspector) oi;
 
         // if writable constant is null then return size 0
-        if (coi.getWritableConstantValue() == null) {
-          return 0;
-        }
-
-        return coi.getWritableConstantValue().toString().length();
-      } else if (oi instanceof WritableConstantHiveVarcharObjectInspector) {
-
-        WritableConstantHiveVarcharObjectInspector wcsoi =
-            (WritableConstantHiveVarcharObjectInspector) oi;
-        return wcsoi.getWritableConstantValue().toString().length();
-      } else if (oi instanceof WritableHiveVarcharObjectInspector) {
-        return ((WritableHiveVarcharObjectInspector) oi).getMaxLength();
+        Object constantValue = coi.getWritableConstantValue();
+        return constantValue == null ? 0 : constantValue.toString().length();
+      } else if (oi instanceof HiveVarcharObjectInspector) {
+        VarcharTypeInfo type = (VarcharTypeInfo) ((HiveVarcharObjectInspector) oi).getTypeInfo();
+        return type.getLength();
       }
     } else if (colType.startsWith(serdeConstants.CHAR_TYPE_NAME)) {
 
@@ -829,18 +810,11 @@ public class StatsUtils {
         ConstantObjectInspector coi = (ConstantObjectInspector) oi;
 
         // if writable constant is null then return size 0
-        if (coi.getWritableConstantValue() == null) {
-          return 0;
-        }
-
-        return coi.getWritableConstantValue().toString().length();
-      } else if (oi instanceof WritableConstantHiveCharObjectInspector) {
-
-        WritableConstantHiveCharObjectInspector wcsoi =
-            (WritableConstantHiveCharObjectInspector) oi;
-        return wcsoi.getWritableConstantValue().toString().length();
-      } else if (oi instanceof WritableHiveCharObjectInspector) {
-        return ((WritableHiveCharObjectInspector) oi).getMaxLength();
+        Object constantValue = coi.getWritableConstantValue();
+        return constantValue == null ? 0 : constantValue.toString().length();
+      } else if (oi instanceof HiveCharObjectInspector) {
+        CharTypeInfo type = (CharTypeInfo) ((HiveCharObjectInspector) oi).getTypeInfo();
+        return type.getLength();
       }
     } else if (colType.equalsIgnoreCase(serdeConstants.BINARY_TYPE_NAME)) {
 
@@ -849,19 +823,9 @@ public class StatsUtils {
         ConstantObjectInspector coi = (ConstantObjectInspector) oi;
 
         // if writable constant is null then return size 0
-        if (coi.getWritableConstantValue() == null) {
-          return 0;
-        }
-
-        BytesWritable bw = ((BytesWritable) coi.getWritableConstantValue());
-        return bw.getLength();
-      } else if (oi instanceof WritableConstantBinaryObjectInspector) {
-
-        // writable constant byte arrays
-        WritableConstantBinaryObjectInspector wcboi = (WritableConstantBinaryObjectInspector) oi;
-
-        return wcboi.getWritableConstantValue().getLength();
-      } else if (oi instanceof WritableBinaryObjectInspector) {
+        BytesWritable constantValue = (BytesWritable)coi.getWritableConstantValue();
+        return constantValue == null ? 0 : constantValue.getLength();
+      } else if (oi instanceof BinaryObjectInspector) {
 
         // return the variable length from config
         return configVarLen;
@@ -1296,6 +1260,8 @@ public class StatsUtils {
       colType = enfd.getTypeString();
       countDistincts = numRows;
       oi = enfd.getWritableObjectInspector();
+    } else {
+      throw new IllegalArgumentException("not supported expr type " + end.getClass());
     }
 
     if (colType.equalsIgnoreCase(serdeConstants.STRING_TYPE_NAME)


[30/50] [abbrv] hive git commit: HIVE-11745 : Alter table Exchange partition with multiple partition_spec is not working (Yongzhi Chen via Szehon)

Posted by xu...@apache.org.
HIVE-11745 : Alter table Exchange partition with multiple partition_spec is not working (Yongzhi Chen via Szehon)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c6895897
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c6895897
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c6895897

Branch: refs/heads/spark
Commit: c6895897bc8178546b10780c5f463353c4ee1b9a
Parents: 5548a9c
Author: Szehon Ho <sz...@cloudera.com>
Authored: Tue Sep 15 16:39:50 2015 -0700
Committer: Szehon Ho <sz...@cloudera.com>
Committed: Tue Sep 15 16:40:26 2015 -0700

----------------------------------------------------------------------
 .../hive/ql/security/FolderPermissionBase.java  |  17 +-
 .../test/resources/testconfiguration.properties |   1 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   6 +
 .../queries/clientpositive/exchgpartition2lel.q |  32 ++++
 .../clientpositive/exchgpartition2lel.q.out     | 182 +++++++++++++++++++
 5 files changed, 237 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c6895897/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/FolderPermissionBase.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/FolderPermissionBase.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/FolderPermissionBase.java
index f28edc6..d98082f 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/FolderPermissionBase.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/FolderPermissionBase.java
@@ -370,7 +370,7 @@ public abstract class FolderPermissionBase {
   }
 
   @Test
-  public void testAlterPartition() throws Exception {
+  public void testPartition() throws Exception {
     String tableName = "alterpart";
     CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key string, value string) partitioned by (part1 int, part2 int, part3 int)");
     Assert.assertEquals(0,ret.getResponseCode());
@@ -396,6 +396,21 @@ public abstract class FolderPermissionBase {
     for (String child : listStatus(warehouseDir + "/" + tableName + "/part1=2/part2=2/part3=2")) {
       verifyPermission(child, 1);
     }
+
+    String tableName2 = "alterpart2";
+    ret = driver.run("CREATE TABLE " + tableName2 + " (key string, value string) partitioned by (part1 int, part2 int, part3 int)");
+    Assert.assertEquals(0,ret.getResponseCode());
+
+    assertExistence(warehouseDir + "/" + tableName2);
+    setPermission(warehouseDir + "/" + tableName2);
+    ret = driver.run("alter table " + tableName2 + " exchange partition (part1='2',part2='2',part3='2') with table " + tableName);
+    Assert.assertEquals(0,ret.getResponseCode());
+
+    //alter exchange can not change base table's permission
+    //alter exchange can only control final partition folder's permission
+    verifyPermission(warehouseDir + "/" + tableName2 + "/part1=2", 0);
+    verifyPermission(warehouseDir + "/" + tableName2 + "/part1=2/part2=2", 0);
+    verifyPermission(warehouseDir + "/" + tableName2 + "/part1=2/part2=2/part3=2", 1);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/c6895897/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index bed621d..2851720 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -12,6 +12,7 @@ minimr.query.files=auto_sortmerge_join_16.q,\
   constprog_partitioner.q,\
   disable_merge_for_bucketing.q,\
   empty_dir_in_table.q,\
+  exchgpartition2lel.q,\
   external_table_with_space_in_location_path.q,\
   file_with_header_footer.q,\
   groupby2.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/c6895897/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 1840e76..a80f686 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -2596,6 +2596,12 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           ms.dropPartition(partition.getDbName(), sourceTable.getTableName(),
             partition.getValues());
         }
+        Path destParentPath = destPath.getParent();
+        if (!wh.isDir(destParentPath)) {
+          if (!wh.mkdirs(destParentPath, true)) {
+              throw new MetaException("Unable to create path " + destParentPath);
+          }
+        }
         /**
          * TODO: Use the hard link feature of hdfs
          * once https://issues.apache.org/jira/browse/HDFS-3370 is done

http://git-wip-us.apache.org/repos/asf/hive/blob/c6895897/ql/src/test/queries/clientpositive/exchgpartition2lel.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/exchgpartition2lel.q b/ql/src/test/queries/clientpositive/exchgpartition2lel.q
new file mode 100644
index 0000000..2b15894
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/exchgpartition2lel.q
@@ -0,0 +1,32 @@
+DROP TABLE IF EXISTS t1;
+DROP TABLE IF EXISTS t2;
+DROP TABLE IF EXISTS t3;
+DROP TABLE IF EXISTS t4;
+
+CREATE TABLE t1 (a int) PARTITIONED BY (d1 int);
+CREATE TABLE t2 (a int) PARTITIONED BY (d1 int);
+CREATE TABLE t3 (a int) PARTITIONED BY (d1 int, d2 int);
+CREATE TABLE t4 (a int) PARTITIONED BY (d1 int, d2 int);
+CREATE TABLE t5 (a int) PARTITIONED BY (d1 int, d2 int, d3 int);
+CREATE TABLE t6 (a int) PARTITIONED BY (d1 int, d2 int, d3 int);
+
+INSERT OVERWRITE TABLE t1 PARTITION (d1 = 1) SELECT key FROM src where key = 100 limit 1;
+INSERT OVERWRITE TABLE t3 PARTITION (d1 = 1, d2 = 1) SELECT key FROM src where key = 100 limit 1;
+INSERT OVERWRITE TABLE t5 PARTITION (d1 = 1, d2 = 1, d3=1) SELECT key FROM src where key = 100 limit 1;
+
+SELECT * FROM t1;
+
+SELECT * FROM t3;
+
+ALTER TABLE t2 EXCHANGE PARTITION (d1 = 1) WITH TABLE t1;
+SELECT * FROM t1;
+SELECT * FROM t2;
+
+ALTER TABLE t4 EXCHANGE PARTITION (d1 = 1, d2 = 1) WITH TABLE t3;
+SELECT * FROM t3;
+SELECT * FROM t4;
+
+ALTER TABLE t6 EXCHANGE PARTITION (d1 = 1, d2 = 1, d3 = 1) WITH TABLE t5;
+SELECT * FROM t5;
+SELECT * FROM t6;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/c6895897/ql/src/test/results/clientpositive/exchgpartition2lel.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/exchgpartition2lel.q.out b/ql/src/test/results/clientpositive/exchgpartition2lel.q.out
new file mode 100644
index 0000000..5997d6b
--- /dev/null
+++ b/ql/src/test/results/clientpositive/exchgpartition2lel.q.out
@@ -0,0 +1,182 @@
+PREHOOK: query: DROP TABLE IF EXISTS t1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE IF EXISTS t1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: DROP TABLE IF EXISTS t2
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE IF EXISTS t2
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: DROP TABLE IF EXISTS t3
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE IF EXISTS t3
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: DROP TABLE IF EXISTS t4
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE IF EXISTS t4
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE t1 (a int) PARTITIONED BY (d1 int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1
+POSTHOOK: query: CREATE TABLE t1 (a int) PARTITIONED BY (d1 int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1
+PREHOOK: query: CREATE TABLE t2 (a int) PARTITIONED BY (d1 int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t2
+POSTHOOK: query: CREATE TABLE t2 (a int) PARTITIONED BY (d1 int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t2
+PREHOOK: query: CREATE TABLE t3 (a int) PARTITIONED BY (d1 int, d2 int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t3
+POSTHOOK: query: CREATE TABLE t3 (a int) PARTITIONED BY (d1 int, d2 int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t3
+PREHOOK: query: CREATE TABLE t4 (a int) PARTITIONED BY (d1 int, d2 int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t4
+POSTHOOK: query: CREATE TABLE t4 (a int) PARTITIONED BY (d1 int, d2 int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t4
+PREHOOK: query: CREATE TABLE t5 (a int) PARTITIONED BY (d1 int, d2 int, d3 int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t5
+POSTHOOK: query: CREATE TABLE t5 (a int) PARTITIONED BY (d1 int, d2 int, d3 int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t5
+PREHOOK: query: CREATE TABLE t6 (a int) PARTITIONED BY (d1 int, d2 int, d3 int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t6
+POSTHOOK: query: CREATE TABLE t6 (a int) PARTITIONED BY (d1 int, d2 int, d3 int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t6
+PREHOOK: query: INSERT OVERWRITE TABLE t1 PARTITION (d1 = 1) SELECT key FROM src where key = 100 limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@t1@d1=1
+POSTHOOK: query: INSERT OVERWRITE TABLE t1 PARTITION (d1 = 1) SELECT key FROM src where key = 100 limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@t1@d1=1
+POSTHOOK: Lineage: t1 PARTITION(d1=1).a EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+PREHOOK: query: INSERT OVERWRITE TABLE t3 PARTITION (d1 = 1, d2 = 1) SELECT key FROM src where key = 100 limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@t3@d1=1/d2=1
+POSTHOOK: query: INSERT OVERWRITE TABLE t3 PARTITION (d1 = 1, d2 = 1) SELECT key FROM src where key = 100 limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@t3@d1=1/d2=1
+POSTHOOK: Lineage: t3 PARTITION(d1=1,d2=1).a EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+PREHOOK: query: INSERT OVERWRITE TABLE t5 PARTITION (d1 = 1, d2 = 1, d3=1) SELECT key FROM src where key = 100 limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@t5@d1=1/d2=1/d3=1
+POSTHOOK: query: INSERT OVERWRITE TABLE t5 PARTITION (d1 = 1, d2 = 1, d3=1) SELECT key FROM src where key = 100 limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@t5@d1=1/d2=1/d3=1
+POSTHOOK: Lineage: t5 PARTITION(d1=1,d2=1,d3=1).a EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+PREHOOK: query: SELECT * FROM t1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t1@d1=1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM t1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t1@d1=1
+#### A masked pattern was here ####
+100	1
+PREHOOK: query: SELECT * FROM t3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t3
+PREHOOK: Input: default@t3@d1=1/d2=1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM t3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t3
+POSTHOOK: Input: default@t3@d1=1/d2=1
+#### A masked pattern was here ####
+100	1	1
+PREHOOK: query: ALTER TABLE t2 EXCHANGE PARTITION (d1 = 1) WITH TABLE t1
+PREHOOK: type: ALTERTABLE_EXCHANGEPARTITION
+POSTHOOK: query: ALTER TABLE t2 EXCHANGE PARTITION (d1 = 1) WITH TABLE t1
+POSTHOOK: type: ALTERTABLE_EXCHANGEPARTITION
+PREHOOK: query: SELECT * FROM t1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM t1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+#### A masked pattern was here ####
+PREHOOK: query: SELECT * FROM t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t2
+PREHOOK: Input: default@t2@d1=1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t2
+POSTHOOK: Input: default@t2@d1=1
+#### A masked pattern was here ####
+100	1
+PREHOOK: query: ALTER TABLE t4 EXCHANGE PARTITION (d1 = 1, d2 = 1) WITH TABLE t3
+PREHOOK: type: ALTERTABLE_EXCHANGEPARTITION
+POSTHOOK: query: ALTER TABLE t4 EXCHANGE PARTITION (d1 = 1, d2 = 1) WITH TABLE t3
+POSTHOOK: type: ALTERTABLE_EXCHANGEPARTITION
+PREHOOK: query: SELECT * FROM t3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t3
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM t3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t3
+#### A masked pattern was here ####
+PREHOOK: query: SELECT * FROM t4
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t4
+PREHOOK: Input: default@t4@d1=1/d2=1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM t4
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t4
+POSTHOOK: Input: default@t4@d1=1/d2=1
+#### A masked pattern was here ####
+100	1	1
+PREHOOK: query: ALTER TABLE t6 EXCHANGE PARTITION (d1 = 1, d2 = 1, d3 = 1) WITH TABLE t5
+PREHOOK: type: ALTERTABLE_EXCHANGEPARTITION
+POSTHOOK: query: ALTER TABLE t6 EXCHANGE PARTITION (d1 = 1, d2 = 1, d3 = 1) WITH TABLE t5
+POSTHOOK: type: ALTERTABLE_EXCHANGEPARTITION
+PREHOOK: query: SELECT * FROM t5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t5
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM t5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t5
+#### A masked pattern was here ####
+PREHOOK: query: SELECT * FROM t6
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t6
+PREHOOK: Input: default@t6@d1=1/d2=1/d3=1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM t6
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t6
+POSTHOOK: Input: default@t6@d1=1/d2=1/d3=1
+#### A masked pattern was here ####
+100	1	1	1


[31/50] [abbrv] hive git commit: HIVE-11821: JDK8 strict build broken for master (Gopal V, reviewed by Sergey Shelukhin)

Posted by xu...@apache.org.
HIVE-11821: JDK8 strict build broken for master (Gopal V, reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bc62a46d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bc62a46d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bc62a46d

Branch: refs/heads/spark
Commit: bc62a46d1a19f343fc74319c52f14a812f3c70fb
Parents: c689589
Author: Gopal V <go...@apache.org>
Authored: Tue Sep 15 17:21:36 2015 -0700
Committer: Gopal V <go...@apache.org>
Committed: Tue Sep 15 17:21:50 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java  | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bc62a46d/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index 8ea1879..c357329 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.optimizer;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -350,8 +351,9 @@ public class ConvertJoinMapJoin implements NodeProcessor {
       // each side better have 0 or more RS. if either side is unbalanced, cannot convert.
       // This is a workaround for now. Right fix would be to refactor code in the
       // MapRecordProcessor and ReduceRecordProcessor with respect to the sources.
+      @SuppressWarnings({"rawtypes","unchecked"})
       Set<ReduceSinkOperator> set =
-          OperatorUtils.findOperatorsUpstream(parentOp.getParentOperators(),
+          OperatorUtils.findOperatorsUpstream((Collection)parentOp.getParentOperators(),
               ReduceSinkOperator.class);
       if (size < 0) {
         size = set.size();


[35/50] [abbrv] hive git commit: HIVE-11832: HIVE-11802 breaks compilation in JDK 8

Posted by xu...@apache.org.
HIVE-11832: HIVE-11802 breaks compilation in JDK 8


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/201b1a00
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/201b1a00
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/201b1a00

Branch: refs/heads/spark
Commit: 201b1a004adbefab1b48098241c0fc43d3dd0dcf
Parents: 5a550cb
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed Sep 16 11:22:27 2015 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed Sep 16 11:22:27 2015 -0500

----------------------------------------------------------------------
 service/src/java/org/apache/hive/service/cli/Column.java | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/201b1a00/service/src/java/org/apache/hive/service/cli/Column.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/Column.java b/service/src/java/org/apache/hive/service/cli/Column.java
index 31091a3..adb269a 100644
--- a/service/src/java/org/apache/hive/service/cli/Column.java
+++ b/service/src/java/org/apache/hive/service/cli/Column.java
@@ -40,7 +40,6 @@ import org.apache.hive.service.cli.thrift.TI16Column;
 import org.apache.hive.service.cli.thrift.TI32Column;
 import org.apache.hive.service.cli.thrift.TI64Column;
 import org.apache.hive.service.cli.thrift.TStringColumn;
-import sun.misc.FloatingDecimal;
 
 /**
  * Column.
@@ -350,7 +349,7 @@ public class Column extends AbstractList {
         break;
       case FLOAT_TYPE:
         nulls.set(size, field == null);
-        doubleVars()[size] = field == null ? 0 : new FloatingDecimal((Float)field).doubleValue();
+        doubleVars()[size] = field == null ? 0 : new Double(field.toString());
         break;
       case DOUBLE_TYPE:
         nulls.set(size, field == null);


[29/50] [abbrv] hive git commit: HIVE-11817: Window function max NullPointerException (Jimmy, reviewed by Szehon)

Posted by xu...@apache.org.
HIVE-11817: Window function max NullPointerException (Jimmy, reviewed by Szehon)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/5548a9c3
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/5548a9c3
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/5548a9c3

Branch: refs/heads/spark
Commit: 5548a9c376c2c59cbad9788689e58daa1a11f21e
Parents: 07ca812
Author: Jimmy Xiang <jx...@cloudera.com>
Authored: Mon Sep 14 12:47:18 2015 -0700
Committer: Jimmy Xiang <jx...@cloudera.com>
Committed: Tue Sep 15 09:14:38 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/udf/generic/GenericUDAFMax.java  | 16 +++++++++-------
 ql/src/test/queries/clientpositive/windowing_udaf.q |  4 ++++
 .../results/clientpositive/windowing_udaf.q.out     | 12 ++++++++++++
 3 files changed, 25 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5548a9c3/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java
index 33600f2..55a6a62 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java
@@ -287,14 +287,16 @@ public class GenericUDAFMax extends AbstractGenericUDAFResolver {
       // For the case: X following and Y following, process first Y-X results and then insert X nulls.
       // For the case X preceding and Y following, process Y results.
       for (int i = Math.max(0, wFrameDef.getStart().getRelativeOffset()); i < wFrameDef.getEnd().getRelativeOffset(); i++) {
-        s.results.add(r[0]);
+        s.results.add(r == null ? null : r[0]);
         s.numRows++;
-        int fIdx = (Integer) r[1];
-        if (!wFrameDef.isStartUnbounded()
-            && s.numRows + i >= fIdx + wFrameDef.getWindowSize()
-            && !s.maxChain.isEmpty()) {
-          s.maxChain.removeFirst();
-          r = !s.maxChain.isEmpty() ? s.maxChain.getFirst() : r;
+        if (r != null) {
+          int fIdx = (Integer) r[1];
+          if (!wFrameDef.isStartUnbounded()
+              && s.numRows + i >= fIdx + wFrameDef.getWindowSize()
+              && !s.maxChain.isEmpty()) {
+            s.maxChain.removeFirst();
+            r = !s.maxChain.isEmpty() ? s.maxChain.getFirst() : r;
+          }
         }
       }
       for (int i = 0; i < wFrameDef.getStart().getRelativeOffset(); i++) {

http://git-wip-us.apache.org/repos/asf/hive/blob/5548a9c3/ql/src/test/queries/clientpositive/windowing_udaf.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/windowing_udaf.q b/ql/src/test/queries/clientpositive/windowing_udaf.q
index 0173ab7..45c5e5b 100644
--- a/ql/src/test/queries/clientpositive/windowing_udaf.q
+++ b/ql/src/test/queries/clientpositive/windowing_udaf.q
@@ -26,3 +26,7 @@ select s, avg(i) over (partition by t, b order by s) from over10k limit 100;
 select max(i) over w from over10k window w as (partition by f) limit 100;
 
 select s, avg(d) over (partition by t order by f) from over10k limit 100;
+
+select key, max(value) over
+  (order by key rows between 10 preceding and 20 following)
+from src1 where length(key) > 10;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/5548a9c3/ql/src/test/results/clientpositive/windowing_udaf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/windowing_udaf.q.out b/ql/src/test/results/clientpositive/windowing_udaf.q.out
index 9d50251..298c1dd 100644
--- a/ql/src/test/results/clientpositive/windowing_udaf.q.out
+++ b/ql/src/test/results/clientpositive/windowing_udaf.q.out
@@ -584,3 +584,15 @@ zach zipper	19.822727272727274
 ulysses king	18.273333333333333
 bob king	17.664615384615384
 luke carson	18.02785714285714
+PREHOOK: query: select key, max(value) over
+  (order by key rows between 10 preceding and 20 following)
+from src1 where length(key) > 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: select key, max(value) over
+  (order by key rows between 10 preceding and 20 following)
+from src1 where length(key) > 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####


[06/50] [abbrv] hive git commit: HIVE-11587 : Fix memory estimates for mapjoin hashtable (Wei Zheng, reviewed by Sergey Shelukhin)

Posted by xu...@apache.org.
HIVE-11587 : Fix memory estimates for mapjoin hashtable (Wei Zheng, reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b4be31f4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b4be31f4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b4be31f4

Branch: refs/heads/spark
Commit: b4be31f4aa497cd09dd1e513eabe951044a7ff73
Parents: 7014407
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Sep 10 11:57:36 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Sep 10 11:57:36 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  2 +
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |  5 ++
 .../persistence/BytesBytesMultiHashMap.java     | 11 +++-
 .../persistence/HybridHashTableContainer.java   | 68 ++++++++++++--------
 .../hive/ql/exec/tez/HashTableLoader.java       |  7 +-
 .../apache/hadoop/hive/serde2/WriteBuffers.java | 10 ++-
 6 files changed, 68 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b4be31f4/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index d2c5885..7f29da2 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -767,6 +767,8 @@ public class HiveConf extends Configuration {
     HIVEMAPJOINUSEOPTIMIZEDTABLE("hive.mapjoin.optimized.hashtable", true,
         "Whether Hive should use memory-optimized hash table for MapJoin. Only works on Tez,\n" +
         "because memory-optimized hashtable cannot be serialized."),
+    HIVEMAPJOINOPTIMIZEDTABLEPROBEPERCENT("hive.mapjoin.optimized.hashtable.probe.percent",
+        (float) 0.5, "Probing space percentage of the optimized hashtable"),
     HIVEUSEHYBRIDGRACEHASHJOIN("hive.mapjoin.hybridgrace.hashtable", true, "Whether to use hybrid" +
         "grace hash join as the join method for mapjoin. Tez only."),
     HIVEHYBRIDGRACEHASHJOINMEMCHECKFREQ("hive.mapjoin.hybridgrace.memcheckfrequency", 1024, "For " +

http://git-wip-us.apache.org/repos/asf/hive/blob/b4be31f4/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
index 1b9d7ef..a9159a5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
@@ -592,6 +592,11 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
 
     // Deserialize the on-disk hash table
     // We're sure this part is smaller than memory limit
+    if (rowCount <= 0) {
+      rowCount = 1024 * 1024; // Since rowCount is used later to instantiate a BytesBytesMultiHashMap
+                              // as the initialCapacity which cannot be 0, we provide a reasonable
+                              // positive number here
+    }
     BytesBytesMultiHashMap restoredHashMap = partition.getHashMapFromDisk(rowCount);
     rowCount += restoredHashMap.getNumValues();
     LOG.info("Hybrid Grace Hash Join: Deserializing spilled hash partition...");

http://git-wip-us.apache.org/repos/asf/hive/blob/b4be31f4/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java
index 3bba890..77c7ead 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java
@@ -153,9 +153,11 @@ public final class BytesBytesMultiHashMap {
   /** 8 Gb of refs is the max capacity if memory limit is not specified. If someone has 100s of
    * Gbs of memory (this might happen pretty soon) we'd need to string together arrays anyway. */
   private final static int DEFAULT_MAX_CAPACITY = 1024 * 1024 * 1024;
+  /** Make sure maxCapacity has a lower limit */
+  private final static int DEFAULT_MIN_MAX_CAPACITY = 16 * 1024 * 1024;
 
   public BytesBytesMultiHashMap(int initialCapacity,
-      float loadFactor, int wbSize, long memUsage) {
+      float loadFactor, int wbSize, long maxProbeSize) {
     if (loadFactor < 0 || loadFactor > 1) {
       throw new AssertionError("Load factor must be between (0, 1].");
     }
@@ -163,8 +165,11 @@ public final class BytesBytesMultiHashMap {
     initialCapacity = (Long.bitCount(initialCapacity) == 1)
         ? initialCapacity : nextHighestPowerOfTwo(initialCapacity);
     // 8 bytes per long in the refs, assume data will be empty. This is just a sanity check.
-    int maxCapacity =  (memUsage <= 0) ? DEFAULT_MAX_CAPACITY
-        : (int)Math.min((long)DEFAULT_MAX_CAPACITY, memUsage / 8);
+    int maxCapacity =  (maxProbeSize <= 0) ? DEFAULT_MAX_CAPACITY
+        : (int)Math.min((long)DEFAULT_MAX_CAPACITY, maxProbeSize / 8);
+    if (maxCapacity < DEFAULT_MIN_MAX_CAPACITY) {
+      maxCapacity = DEFAULT_MIN_MAX_CAPACITY;
+    }
     if (maxCapacity < initialCapacity || initialCapacity <= 0) {
       // Either initialCapacity is too large, or nextHighestPowerOfTwo overflows
       initialCapacity = (Long.bitCount(maxCapacity) == 1)

http://git-wip-us.apache.org/repos/asf/hive/blob/b4be31f4/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
index ff64f52..52c02ae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
@@ -76,7 +76,6 @@ public class HybridHashTableContainer
   private int totalInMemRowCount = 0;           // total number of small table rows in memory
   private long memoryThreshold;                 // the max memory limit that can be allocated
   private long memoryUsed;                      // the actual memory used
-  private int writeBufferSize;                  // write buffer size for this HybridHashTableContainer
   private final long tableRowSize;              // row size of the small table
   private boolean isSpilled;                    // whether there's any spilled partition
   private int toSpillPartitionId;               // the partition into which to spill the big table row;
@@ -107,7 +106,7 @@ public class HybridHashTableContainer
     Path hashMapLocalPath;                  // Local file system path for spilled hashMap
     boolean hashMapOnDisk;                  // Status of hashMap. true: on disk, false: in memory
     boolean hashMapSpilledOnCreation;       // When there's no enough memory, cannot create hashMap
-    int threshold;                          // Used to create an empty BytesBytesMultiHashMap
+    int initialCapacity;                    // Used to create an empty BytesBytesMultiHashMap
     float loadFactor;                       // Same as above
     int wbSize;                             // Same as above
     int rowsOnDisk;                         // How many rows saved to the on-disk hashmap (if on disk)
@@ -115,17 +114,17 @@ public class HybridHashTableContainer
     /* It may happen that there's not enough memory to instantiate a hashmap for the partition.
      * In that case, we don't create the hashmap, but pretend the hashmap is directly "spilled".
      */
-    public HashPartition(int threshold, float loadFactor, int wbSize, long memUsage,
+    public HashPartition(int initialCapacity, float loadFactor, int wbSize, long maxProbeSize,
                          boolean createHashMap) {
       if (createHashMap) {
-        // Hash map should be at least the size of our designated wbSize
-        memUsage = Math.max(memUsage, wbSize);
-        hashMap = new BytesBytesMultiHashMap(threshold, loadFactor, wbSize, memUsage);
+        // Probe space should be at least equal to the size of our designated wbSize
+        maxProbeSize = Math.max(maxProbeSize, wbSize);
+        hashMap = new BytesBytesMultiHashMap(initialCapacity, loadFactor, wbSize, maxProbeSize);
       } else {
         hashMapSpilledOnCreation = true;
         hashMapOnDisk = true;
       }
-      this.threshold = threshold;
+      this.initialCapacity = initialCapacity;
       this.loadFactor = loadFactor;
       this.wbSize = wbSize;
     }
@@ -138,18 +137,18 @@ public class HybridHashTableContainer
     /* Restore the hashmap from disk by deserializing it.
      * Currently Kryo is used for this purpose.
      */
-    public BytesBytesMultiHashMap getHashMapFromDisk(int initialCapacity)
+    public BytesBytesMultiHashMap getHashMapFromDisk(int rowCount)
         throws IOException, ClassNotFoundException {
       if (hashMapSpilledOnCreation) {
-        return new BytesBytesMultiHashMap(Math.max(threshold, initialCapacity) , loadFactor, wbSize, -1);
+        return new BytesBytesMultiHashMap(rowCount, loadFactor, wbSize, -1);
       } else {
         InputStream inputStream = Files.newInputStream(hashMapLocalPath);
         com.esotericsoftware.kryo.io.Input input = new com.esotericsoftware.kryo.io.Input(inputStream);
         Kryo kryo = Utilities.runtimeSerializationKryo.get();
         BytesBytesMultiHashMap restoredHashMap = kryo.readObject(input, BytesBytesMultiHashMap.class);
 
-        if (initialCapacity > 0) {
-          restoredHashMap.expandAndRehashToTarget(initialCapacity);
+        if (rowCount > 0) {
+          restoredHashMap.expandAndRehashToTarget(rowCount);
         }
 
         // some bookkeeping
@@ -237,7 +236,7 @@ public class HybridHashTableContainer
 
   public HybridHashTableContainer(Configuration hconf, long keyCount, long memoryAvailable,
                                   long estimatedTableSize, HybridHashTableConf nwayConf)
- throws SerDeException, IOException {
+      throws SerDeException, IOException {
     this(HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEHASHTABLEKEYCOUNTADJUSTMENT),
         HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHASHTABLETHRESHOLD),
         HiveConf.getFloatVar(hconf,HiveConf.ConfVars.HIVEHASHTABLELOADFACTOR),
@@ -245,12 +244,13 @@ public class HybridHashTableContainer
         HiveConf.getIntVar(hconf,HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINWBSIZE),
         HiveConf.getIntVar(hconf,HiveConf.ConfVars.HIVEHASHTABLEWBSIZE),
         HiveConf.getIntVar(hconf,HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINNUMPARTITIONS),
+        HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEMAPJOINOPTIMIZEDTABLEPROBEPERCENT),
         estimatedTableSize, keyCount, memoryAvailable, nwayConf);
   }
 
   private HybridHashTableContainer(float keyCountAdj, int threshold, float loadFactor,
-      int memCheckFreq, int minWbSize, int maxWbSize, int minNumParts, long estimatedTableSize,
-      long keyCount, long memoryAvailable, HybridHashTableConf nwayConf)
+      int memCheckFreq, int minWbSize, int maxWbSize, int minNumParts, float probePercent,
+      long estimatedTableSize, long keyCount, long memoryAvailable, HybridHashTableConf nwayConf)
       throws SerDeException, IOException {
     directWriteHelper = new MapJoinBytesTableContainer.DirectKeyValueWriter();
 
@@ -262,10 +262,10 @@ public class HybridHashTableContainer
     memoryCheckFrequency = memCheckFreq;
 
     this.nwayConf = nwayConf;
+    int writeBufferSize;
     int numPartitions;
     if (nwayConf == null) { // binary join
-      numPartitions = calcNumPartitions(memoryThreshold, estimatedTableSize, minNumParts, minWbSize,
-          nwayConf);
+      numPartitions = calcNumPartitions(memoryThreshold, estimatedTableSize, minNumParts, minWbSize);
       writeBufferSize = (int)(estimatedTableSize / numPartitions);
     } else {                // n-way join
       // It has been calculated in HashTableLoader earlier, so just need to retrieve that number
@@ -302,21 +302,33 @@ public class HybridHashTableContainer
     int numPartitionsSpilledOnCreation = 0;
     memoryUsed = 0;
     int initialCapacity = Math.max(newKeyCount / numPartitions, threshold / numPartitions);
+    // maxCapacity should be calculated based on a percentage of memoryThreshold, which is to divide
+    // row size using long size
+    float probePercentage = (float) 8 / (tableRowSize + 8); // long_size / tableRowSize + long_size
+    if (probePercentage == 1) {
+      probePercentage = probePercent;
+    }
+    int maxCapacity = (int)(memoryThreshold * probePercentage);
     for (int i = 0; i < numPartitions; i++) {
       if (this.nwayConf == null ||                          // binary join
           nwayConf.getLoadedContainerList().size() == 0) {  // n-way join, first (biggest) small table
         if (i == 0) { // We unconditionally create a hashmap for the first hash partition
-          hashPartitions[i] = new HashPartition(initialCapacity, loadFactor, writeBufferSize, memoryThreshold, true);
+          hashPartitions[i] = new HashPartition(initialCapacity, loadFactor, writeBufferSize,
+              maxCapacity, true);
         } else {
-          hashPartitions[i] = new HashPartition(initialCapacity, loadFactor, writeBufferSize, memoryThreshold,
-              memoryUsed + writeBufferSize < memoryThreshold);
+          // To check whether we have enough memory to allocate for another hash partition,
+          // we need to get the size of the first hash partition to get an idea.
+          hashPartitions[i] = new HashPartition(initialCapacity, loadFactor, writeBufferSize,
+              maxCapacity, memoryUsed + hashPartitions[0].hashMap.memorySize() < memoryThreshold);
         }
-      } else {                      // n-way join
+      } else {                                              // n-way join, all later small tables
         // For all later small tables, follow the same pattern of the previously loaded tables.
         if (this.nwayConf.doSpillOnCreation(i)) {
-          hashPartitions[i] = new HashPartition(threshold, loadFactor, writeBufferSize, memoryThreshold, false);
+          hashPartitions[i] = new HashPartition(initialCapacity, loadFactor, writeBufferSize,
+              maxCapacity, false);
         } else {
-          hashPartitions[i] = new HashPartition(threshold, loadFactor, writeBufferSize, memoryThreshold, true);
+          hashPartitions[i] = new HashPartition(initialCapacity, loadFactor, writeBufferSize,
+              maxCapacity, true);
         }
       }
 
@@ -513,7 +525,8 @@ public class HybridHashTableContainer
     Path path = Files.createTempFile("partition-" + partitionId + "-", null);
     OutputStream outputStream = Files.newOutputStream(path);
 
-    com.esotericsoftware.kryo.io.Output output = new com.esotericsoftware.kryo.io.Output(outputStream);
+    com.esotericsoftware.kryo.io.Output output =
+        new com.esotericsoftware.kryo.io.Output(outputStream);
     Kryo kryo = Utilities.runtimeSerializationKryo.get();
     kryo.writeObject(output, partition.hashMap);  // use Kryo to serialize hashmap
     output.close();
@@ -545,11 +558,10 @@ public class HybridHashTableContainer
    * @param dataSize total data size for the table
    * @param minNumParts minimum required number of partitions
    * @param minWbSize minimum required write buffer size
-   * @param nwayConf the n-way join configuration
    * @return number of partitions needed
    */
   public static int calcNumPartitions(long memoryThreshold, long dataSize, int minNumParts,
-      int minWbSize, HybridHashTableConf nwayConf) throws IOException {
+      int minWbSize) throws IOException {
     int numPartitions = minNumParts;
 
     if (memoryThreshold < minNumParts * minWbSize) {
@@ -803,7 +815,8 @@ public class HybridHashTableContainer
         return JoinUtil.JoinResult.SPILL;
       }
       else {
-        aliasFilter = hashPartitions[partitionId].hashMap.getValueResult(output.getData(), 0, output.getLength(), hashMapResult);
+        aliasFilter = hashPartitions[partitionId].hashMap.getValueResult(output.getData(), 0,
+            output.getLength(), hashMapResult);
         dummyRow = null;
         if (hashMapResult.hasRows()) {
           return JoinUtil.JoinResult.MATCH;
@@ -941,7 +954,8 @@ public class HybridHashTableContainer
         return JoinUtil.JoinResult.SPILL;
       }
       else {
-        aliasFilter = hashPartitions[partitionId].hashMap.getValueResult(bytes, offset, length, hashMapResult);
+        aliasFilter = hashPartitions[partitionId].hashMap.getValueResult(bytes, offset, length,
+            hashMapResult);
         dummyRow = null;
         if (hashMapResult.hasRows()) {
           return JoinUtil.JoinResult.MATCH;

http://git-wip-us.apache.org/repos/asf/hive/blob/b4be31f4/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java
index 2b6571b..f7d165a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java
@@ -84,6 +84,7 @@ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTable
 
     // Get the total available memory from memory manager
     long totalMapJoinMemory = desc.getMemoryNeeded();
+    LOG.info("Memory manager allocates " + totalMapJoinMemory + " bytes for the loading hashtable.");
     if (totalMapJoinMemory <= 0) {
       totalMapJoinMemory = HiveConf.getLongVar(
         hconf, HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD);
@@ -128,11 +129,9 @@ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTable
       long memory = tableMemorySizes.get(biggest);
       int numPartitions = 0;
       try {
-        numPartitions = HybridHashTableContainer.calcNumPartitions(memory,
-            maxSize,
+        numPartitions = HybridHashTableContainer.calcNumPartitions(memory, maxSize,
             HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINNUMPARTITIONS),
-            HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINWBSIZE),
-            nwayConf);
+            HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINWBSIZE));
       } catch (IOException e) {
         throw new HiveException(e);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/b4be31f4/serde/src/java/org/apache/hadoop/hive/serde2/WriteBuffers.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/WriteBuffers.java b/serde/src/java/org/apache/hadoop/hive/serde2/WriteBuffers.java
index 05d9359..62250ec 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/WriteBuffers.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/WriteBuffers.java
@@ -61,7 +61,6 @@ public final class WriteBuffers implements RandomAccessOutput {
     this.offsetMask = this.wbSize - 1;
     this.maxSize = maxSize;
     writePos.bufferIndex = -1;
-    nextBufferToWrite();
   }
 
   public int readVInt() {
@@ -207,6 +206,9 @@ public final class WriteBuffers implements RandomAccessOutput {
 
   @Override
   public void write(byte[] b, int off, int len) {
+    if (writePos.bufferIndex == -1) {
+      nextBufferToWrite();
+    }
     int srcOffset = 0;
     while (srcOffset < len) {
       int toWrite = Math.min(len - srcOffset, wbSize - writePos.offset);
@@ -355,6 +357,9 @@ public final class WriteBuffers implements RandomAccessOutput {
 
 
   public long getWritePoint() {
+    if (writePos.bufferIndex == -1) {
+      nextBufferToWrite();
+    }
     return ((long)writePos.bufferIndex << wbSizeLog2) + writePos.offset;
   }
 
@@ -498,6 +503,9 @@ public final class WriteBuffers implements RandomAccessOutput {
   }
 
   public void seal() {
+    if (writePos.bufferIndex == -1) {
+      return;
+    }
     if (writePos.offset < (wbSize * 0.8)) { // arbitrary
       byte[] smallerBuffer = new byte[writePos.offset];
       System.arraycopy(writePos.buffer, 0, smallerBuffer, 0, writePos.offset);


[26/50] [abbrv] hive git commit: HIVE-11544: Improve LazySimpleSerDe null data handling for Byte, Short, Integer, Float, Long and Double. (Gopal V, reviewed by Ashutosh Chauhan)

Posted by xu...@apache.org.
HIVE-11544: Improve LazySimpleSerDe null data handling for Byte, Short, Integer, Float, Long and Double. (Gopal V, reviewed by Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/98049182
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/98049182
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/98049182

Branch: refs/heads/spark
Commit: 980491821e6c33d595e8a0e3abcfbc6c207aa436
Parents: 0bc9677
Author: Gopal V <go...@apache.org>
Authored: Mon Sep 14 18:13:42 2015 -0700
Committer: Gopal V <go...@apache.org>
Committed: Mon Sep 14 18:13:42 2015 -0700

----------------------------------------------------------------------
 .../benchmark/serde/LazySimpleSerDeBench.java   | 453 +++++++++++++++++++
 .../hadoop/hive/serde2/lazy/LazyByte.java       |   4 +
 .../hadoop/hive/serde2/lazy/LazyDouble.java     |   4 +
 .../hadoop/hive/serde2/lazy/LazyFloat.java      |   4 +
 .../hadoop/hive/serde2/lazy/LazyInteger.java    |   4 +
 .../hadoop/hive/serde2/lazy/LazyLong.java       |   4 +
 .../hadoop/hive/serde2/lazy/LazyShort.java      |   4 +
 .../hadoop/hive/serde2/lazy/LazyUtils.java      |  28 ++
 8 files changed, 505 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/98049182/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/serde/LazySimpleSerDeBench.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/serde/LazySimpleSerDeBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/serde/LazySimpleSerDeBench.java
new file mode 100644
index 0000000..a1b63d5
--- /dev/null
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/serde/LazySimpleSerDeBench.java
@@ -0,0 +1,453 @@
+/*
+ * Licensed 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.hive.benchmark.serde;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
+import org.apache.hadoop.hive.serde2.lazy.LazyByte;
+import org.apache.hadoop.hive.serde2.lazy.LazyDouble;
+import org.apache.hadoop.hive.serde2.lazy.LazyFloat;
+import org.apache.hadoop.hive.serde2.lazy.LazyInteger;
+import org.apache.hadoop.hive.serde2.lazy.LazyLong;
+import org.apache.hadoop.hive.serde2.lazy.LazyShort;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyPrimitiveObjectInspectorFactory;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+@State(Scope.Benchmark)
+public class LazySimpleSerDeBench {
+  /**
+   * This test measures the performance for LazySimpleSerDe.
+   * <p/>
+   * This test uses JMH framework for benchmarking. You may execute this
+   * benchmark tool using JMH command line in different ways:
+   * <p/>
+   * To run using default settings, use: 
+   * $ java -cp target/benchmarks.jar org.apache.hive.benchmark.serde.LazySimpleSerDeBench
+   * <p/>
+   */
+
+  @BenchmarkMode(Mode.AverageTime)
+  @Fork(1)
+  @State(Scope.Thread)
+  @OutputTimeUnit(TimeUnit.NANOSECONDS)
+  public static abstract class AbstractDeserializer {
+    public static final int DEFAULT_ITER_TIME = 1000000;
+
+    public static final int DEFAULT_DATA_SIZE = 4096;
+
+    public int[] offsets = new int[DEFAULT_DATA_SIZE];
+    public int[] sizes = new int[DEFAULT_DATA_SIZE];
+    protected final ByteArrayRef ref = new ByteArrayRef();
+
+    @Setup
+    public abstract void setup();
+
+    @Benchmark
+    @Warmup(iterations = 2, time = 2, timeUnit = TimeUnit.MILLISECONDS)
+    @Measurement(iterations = 2, time = 2, timeUnit = TimeUnit.MILLISECONDS)
+    public void bench() {
+
+    }
+  }
+
+  public static abstract class RandomDataInitializer extends
+      AbstractDeserializer {
+
+    final int width;
+
+    public RandomDataInitializer(final int width) {
+      this.width = width;
+    }
+
+    @Override
+    public void setup() {
+      int len = 0;
+      Random r = new Random();
+      for (int i = 0; i < sizes.length; i++) {
+        sizes[i] = (int) (r.nextInt(width));
+        offsets[i] = len;
+        len += sizes[i];
+      }
+      byte[] data = new byte[len + 1];
+      r.nextBytes(data);
+      ref.setData(data);
+    }
+  }
+
+  public static abstract class GoodDataInitializer extends AbstractDeserializer {
+
+    public final int max;
+
+    public GoodDataInitializer(final int max) {
+      this.max = max;
+    }
+
+    @Override
+    public void setup() {
+      sizes = new int[1024];
+      offsets = new int[sizes.length];
+      ByteArrayOutputStream bos = new ByteArrayOutputStream();
+      Random r = new Random();
+      int len = 0;
+      for (int i = 0; i < sizes.length / 2; i++) {
+        int p = r.nextInt(max);
+        int n = -1 * (p - 1);
+        byte[] ps = String.format("%d", p).getBytes();
+        byte[] ns = String.format("%d", n).getBytes();
+        sizes[2 * i] = ps.length;
+        sizes[2 * i + 1] = ns.length;
+        offsets[2 * i] = len;
+        offsets[2 * i + 1] = len + ps.length;
+        len += ps.length + ns.length;
+        try {
+          bos.write(ns);
+          bos.write(ps);
+        } catch (IOException e) {
+          e.printStackTrace();
+          throw new RuntimeException(e);
+        }
+      }
+      ref.setData(bos.toByteArray());
+    }
+  }
+
+  public static class RandomLazyByte extends RandomDataInitializer {
+
+    public RandomLazyByte() {
+      super(2);
+    }
+
+    final LazyByte obj = new LazyByte(
+        LazyPrimitiveObjectInspectorFactory.LAZY_BYTE_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class WorstLazyByte extends RandomDataInitializer {
+
+    public WorstLazyByte() {
+      super(8);
+    }
+
+    final LazyByte obj = new LazyByte(
+        LazyPrimitiveObjectInspectorFactory.LAZY_BYTE_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class GoodLazyByte extends GoodDataInitializer {
+
+    final LazyByte obj = new LazyByte(
+        LazyPrimitiveObjectInspectorFactory.LAZY_BYTE_OBJECT_INSPECTOR);
+
+    public GoodLazyByte() {
+      super(Integer.MAX_VALUE);
+    }
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class RandomLazyShort extends RandomDataInitializer {
+
+    public RandomLazyShort() {
+      super(2);
+    }
+
+    final LazyShort obj = new LazyShort(
+        LazyPrimitiveObjectInspectorFactory.LAZY_SHORT_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class WorstLazyShort extends RandomDataInitializer {
+
+    public WorstLazyShort() {
+      super(8);
+    }
+
+    final LazyShort obj = new LazyShort(
+        LazyPrimitiveObjectInspectorFactory.LAZY_SHORT_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class GoodLazyShort extends GoodDataInitializer {
+
+    final LazyShort obj = new LazyShort(
+        LazyPrimitiveObjectInspectorFactory.LAZY_SHORT_OBJECT_INSPECTOR);
+
+    public GoodLazyShort() {
+      super(Integer.MAX_VALUE);
+    }
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class RandomLazyInteger extends RandomDataInitializer {
+
+    public RandomLazyInteger() {
+      super(2);
+    }
+
+    final LazyInteger obj = new LazyInteger(
+        LazyPrimitiveObjectInspectorFactory.LAZY_INT_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class WorstLazyInteger extends RandomDataInitializer {
+
+    public WorstLazyInteger() {
+      super(8);
+    }
+
+    final LazyInteger obj = new LazyInteger(
+        LazyPrimitiveObjectInspectorFactory.LAZY_INT_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class GoodLazyInteger extends GoodDataInitializer {
+
+    final LazyInteger obj = new LazyInteger(
+        LazyPrimitiveObjectInspectorFactory.LAZY_INT_OBJECT_INSPECTOR);
+
+    public GoodLazyInteger() {
+      super(Integer.MAX_VALUE);
+    }
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class RandomLazyFloat extends RandomDataInitializer {
+
+    public RandomLazyFloat() {
+      super(2);
+    }
+
+    final LazyFloat obj = new LazyFloat(
+        LazyPrimitiveObjectInspectorFactory.LAZY_FLOAT_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class WorstLazyFloat extends RandomDataInitializer {
+
+    public WorstLazyFloat() {
+      super(8);
+    }
+
+    final LazyFloat obj = new LazyFloat(
+        LazyPrimitiveObjectInspectorFactory.LAZY_FLOAT_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class GoodLazyFloat extends GoodDataInitializer {
+
+    final LazyFloat obj = new LazyFloat(
+        LazyPrimitiveObjectInspectorFactory.LAZY_FLOAT_OBJECT_INSPECTOR);
+
+    public GoodLazyFloat() {
+      super(Integer.MAX_VALUE);
+    }
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class RandomLazyLong extends RandomDataInitializer {
+
+    public RandomLazyLong() {
+      super(2);
+    }
+
+    final LazyLong obj = new LazyLong(
+        LazyPrimitiveObjectInspectorFactory.LAZY_LONG_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class WorstLazyLong extends RandomDataInitializer {
+
+    public WorstLazyLong() {
+      super(8);
+    }
+
+    final LazyLong obj = new LazyLong(
+        LazyPrimitiveObjectInspectorFactory.LAZY_LONG_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class GoodLazyLong extends GoodDataInitializer {
+
+    final LazyLong obj = new LazyLong(
+        LazyPrimitiveObjectInspectorFactory.LAZY_LONG_OBJECT_INSPECTOR);
+
+    public GoodLazyLong() {
+      super(Integer.MAX_VALUE);
+    }
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class RandomLazyDouble extends RandomDataInitializer {
+
+    public RandomLazyDouble() {
+      super(2);
+    }
+
+    final LazyDouble obj = new LazyDouble(
+        LazyPrimitiveObjectInspectorFactory.LAZY_DOUBLE_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class WorstLazyDouble extends RandomDataInitializer {
+
+    public WorstLazyDouble() {
+      super(8);
+    }
+
+    final LazyDouble obj = new LazyDouble(
+        LazyPrimitiveObjectInspectorFactory.LAZY_DOUBLE_OBJECT_INSPECTOR);
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static class GoodLazyDouble extends GoodDataInitializer {
+
+    final LazyDouble obj = new LazyDouble(
+        LazyPrimitiveObjectInspectorFactory.LAZY_DOUBLE_OBJECT_INSPECTOR);
+
+    public GoodLazyDouble() {
+      super(Integer.MAX_VALUE);
+    }
+
+    @Override
+    public void bench() {
+      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+        obj.init(ref, offsets[i % sizes.length], sizes[i % sizes.length]);
+      }
+    }
+  }
+
+  public static void main(String[] args) throws RunnerException {
+    Options opt = new OptionsBuilder().include(
+        ".*" + LazySimpleSerDeBench.class.getSimpleName() + ".*").build();
+    new Runner(opt).run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/98049182/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyByte.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyByte.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyByte.java
index a3b8f76..1f9cead 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyByte.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyByte.java
@@ -48,6 +48,10 @@ public class LazyByte extends
 
   @Override
   public void init(ByteArrayRef bytes, int start, int length) {
+    if (!LazyUtils.isNumberMaybe(bytes.getData(), start, length)) {
+      isNull = true;
+      return;
+    }
     try {
       data.set(parseByte(bytes.getData(), start, length, 10));
       isNull = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/98049182/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDouble.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDouble.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDouble.java
index 05ca4e9..35c2141 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDouble.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDouble.java
@@ -46,6 +46,10 @@ public class LazyDouble extends
   @Override
   public void init(ByteArrayRef bytes, int start, int length) {
     String byteData = null;
+    if (!LazyUtils.isNumberMaybe(bytes.getData(), start, length)) {
+      isNull = true;
+      return;
+    }
     try {
       byteData = Text.decode(bytes.getData(), start, length);
       data.set(Double.parseDouble(byteData));

http://git-wip-us.apache.org/repos/asf/hive/blob/98049182/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFloat.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFloat.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFloat.java
index 37676d1..6e132c7 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFloat.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFloat.java
@@ -46,6 +46,10 @@ public class LazyFloat extends
   @Override
   public void init(ByteArrayRef bytes, int start, int length) {
     String byteData = null;
+    if (!LazyUtils.isNumberMaybe(bytes.getData(), start, length)) {
+      isNull = true;
+      return;
+    }
     try {
       byteData = Text.decode(bytes.getData(), start, length);
       data.set(Float.parseFloat(byteData));

http://git-wip-us.apache.org/repos/asf/hive/blob/98049182/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyInteger.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyInteger.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyInteger.java
index ad82ebf..22742aa 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyInteger.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyInteger.java
@@ -51,6 +51,10 @@ public class LazyInteger extends
 
   @Override
   public void init(ByteArrayRef bytes, int start, int length) {
+    if (!LazyUtils.isNumberMaybe(bytes.getData(), start, length)) {
+      isNull = true;
+      return;
+    }
     try {
       data.set(parseInt(bytes.getData(), start, length, 10));
       isNull = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/98049182/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyLong.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyLong.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyLong.java
index a9779a0..c0d52b9 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyLong.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyLong.java
@@ -51,6 +51,10 @@ public class LazyLong extends
 
   @Override
   public void init(ByteArrayRef bytes, int start, int length) {
+    if (!LazyUtils.isNumberMaybe(bytes.getData(), start, length)) {
+      isNull = true;
+      return;
+    }
     try {
       data.set(parseLong(bytes.getData(), start, length, 10));
       isNull = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/98049182/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyShort.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyShort.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyShort.java
index f04e131..b8b9488 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyShort.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyShort.java
@@ -48,6 +48,10 @@ public class LazyShort extends
 
   @Override
   public void init(ByteArrayRef bytes, int start, int length) {
+    if (!LazyUtils.isNumberMaybe(bytes.getData(), start, length)) {
+      isNull = true;
+      return;
+    }
     try {
       data.set(parseShort(bytes.getData(), start, length));
       isNull = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/98049182/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
index 5c58f6b..a5e4be4 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
@@ -81,6 +81,34 @@ public final class LazyUtils {
   }
 
   /**
+   * returns false, when the bytes definitely cannot be parsed into a base-10
+   * Number (Long or a Double)
+   * 
+   * If it returns true, the bytes might still be invalid, but not obviously.
+   */
+
+  public static boolean isNumberMaybe(byte[] buf, int offset, int len) {
+    switch (len) {
+    case 0:
+      return false;
+    case 1:
+      // space usually
+      return Character.isDigit(buf[offset]);
+    case 2:
+      // \N or -1 (allow latter)
+      return Character.isDigit(buf[offset + 1])
+          || Character.isDigit(buf[offset + 0]);
+    case 4:
+      // null or NULL
+      if (buf[offset] == 'N' || buf[offset] == 'n') {
+        return false;
+      }
+    }
+    // maybe valid - too expensive to check without a parse
+    return true;
+  }
+
+  /**
    * Returns -1 if the first byte sequence is lexicographically less than the
    * second; returns +1 if the second byte sequence is lexicographically less
    * than the first; otherwise return 0.


[17/50] [abbrv] hive git commit: HIVE-11763: Use * instead of sum(hash(*)) on Parquet predicate (PPD) integration tests (Sergio Pena, reviewed by Ferdinand Xu)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/results/clientpositive/parquet_ppd_timestamp.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_ppd_timestamp.q.out b/ql/src/test/results/clientpositive/parquet_ppd_timestamp.q.out
index 745237d..3693879 100644
--- a/ql/src/test/results/clientpositive/parquet_ppd_timestamp.q.out
+++ b/ql/src/test/results/clientpositive/parquet_ppd_timestamp.q.out
@@ -6,11 +6,11 @@ POSTHOOK: query: create table newtypestbl(c char(10), v varchar(10), d decimal(5
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@newtypestbl
-PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("2011-01-01 01:01:01" as timestamp) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("2011-01-20 01:01:01" as timestamp) from src src2) uniontbl
+PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("2011-01-01 01:01:01" as timestamp) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("2011-01-20 01:01:01" as timestamp) from src src2 limit 10) uniontbl
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@newtypestbl
-POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("2011-01-01 01:01:01" as timestamp) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("2011-01-20 01:01:01" as timestamp) from src src2) uniontbl
+POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("2011-01-01 01:01:01" as timestamp) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("2011-01-20 01:01:01" as timestamp) from src src2 limit 10) uniontbl
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@newtypestbl
@@ -19,274 +19,404 @@ POSTHOOK: Lineage: newtypestbl.d EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.ts EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.v EXPRESSION []
 PREHOOK: query: -- timestamp data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where cast(ts as string)='2011-01-01 01:01:01'
+select * from newtypestbl where cast(ts as string)='2011-01-01 01:01:01'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
 POSTHOOK: query: -- timestamp data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where cast(ts as string)='2011-01-01 01:01:01'
+select * from newtypestbl where cast(ts as string)='2011-01-01 01:01:01'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where cast(ts as string)='2011-01-01 01:01:01'
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where cast(ts as string)='2011-01-01 01:01:01'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where cast(ts as string)='2011-01-01 01:01:01'
+POSTHOOK: query: select * from newtypestbl where cast(ts as string)='2011-01-01 01:01:01'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp)
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp)
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20))
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20))
+POSTHOOK: query: select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20))
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20))
+POSTHOOK: query: select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp)
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1033237945500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp)
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+PREHOOK: query: select * from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1033237945500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp)
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+PREHOOK: query: select * from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp)
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp)
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1478890961000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp)
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1478890961000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp)
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp)
+PREHOOK: query: select * from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp)
+PREHOOK: query: select * from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp)
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp)
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1478890961000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp)
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1478890961000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
+POSTHOOK: query: select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1033237945500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+PREHOOK: query: select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
+POSTHOOK: query: select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1033237945500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+PREHOOK: query: select * from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
+POSTHOOK: query: select * from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1478890961000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
+POSTHOOK: query: select * from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1478890961000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp))
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp))
+POSTHOOK: query: select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp))
+PREHOOK: query: select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp))
+POSTHOOK: query: select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp)
+PREHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp)
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-445653015500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp)
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1478890961000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp)
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-1478890961000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp)
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+hello     	world	11.22	2011-01-20 01:01:01
+apple     	bee	0.22	2011-01-01 01:01:01
+PREHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp)
+PREHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp)
+POSTHOOK: query: select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/results/clientpositive/parquet_ppd_varchar.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_ppd_varchar.q.out b/ql/src/test/results/clientpositive/parquet_ppd_varchar.q.out
index 23e3cd0..0574e5d 100644
--- a/ql/src/test/results/clientpositive/parquet_ppd_varchar.q.out
+++ b/ql/src/test/results/clientpositive/parquet_ppd_varchar.q.out
@@ -6,11 +6,11 @@ POSTHOOK: query: create table newtypestbl(c char(10), v varchar(10), d decimal(5
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@newtypestbl
-PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl
+PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@newtypestbl
-POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl
+POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@newtypestbl
@@ -19,202 +19,290 @@ POSTHOOK: Lineage: newtypestbl.d EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.da EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.v EXPRESSION []
 PREHOOK: query: -- varchar data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where v="bee"
+select * from newtypestbl where v="bee"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
 POSTHOOK: query: -- varchar data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where v="bee"
+select * from newtypestbl where v="bee"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v="bee"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v="bee"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v="bee"
+POSTHOOK: query: select * from newtypestbl where v="bee"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v!="bee"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v!="bee"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v!="bee"
+POSTHOOK: query: select * from newtypestbl where v!="bee"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v!="bee"
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where v!="bee"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v!="bee"
+POSTHOOK: query: select * from newtypestbl where v!="bee"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v<"world"
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where v<"world"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v<"world"
+POSTHOOK: query: select * from newtypestbl where v<"world"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v<"world"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v<"world"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v<"world"
+POSTHOOK: query: select * from newtypestbl where v<"world"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v<="world"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v<="world"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v<="world"
+POSTHOOK: query: select * from newtypestbl where v<="world"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v<="world"
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v<="world"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v<="world"
+POSTHOOK: query: select * from newtypestbl where v<="world"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v="bee   "
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v="bee   "
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v="bee   "
+POSTHOOK: query: select * from newtypestbl where v="bee   "
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v="bee   "
+PREHOOK: query: select * from newtypestbl where v="bee   "
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v="bee   "
+POSTHOOK: query: select * from newtypestbl where v="bee   "
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v in ("bee", "orange")
+PREHOOK: query: select * from newtypestbl where v in ("bee", "orange")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v in ("bee", "orange")
+POSTHOOK: query: select * from newtypestbl where v in ("bee", "orange")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v in ("bee", "orange")
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v in ("bee", "orange")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v in ("bee", "orange")
+POSTHOOK: query: select * from newtypestbl where v in ("bee", "orange")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v in ("bee", "world")
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v in ("bee", "world")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v in ("bee", "world")
+POSTHOOK: query: select * from newtypestbl where v in ("bee", "world")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v in ("bee", "world")
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v in ("bee", "world")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v in ("bee", "world")
+POSTHOOK: query: select * from newtypestbl where v in ("bee", "world")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v in ("orange")
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v in ("orange")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v in ("orange")
+POSTHOOK: query: select * from newtypestbl where v in ("orange")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v in ("orange")
+PREHOOK: query: select * from newtypestbl where v in ("orange")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v in ("orange")
+POSTHOOK: query: select * from newtypestbl where v in ("orange")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v between "bee" and "orange"
+PREHOOK: query: select * from newtypestbl where v between "bee" and "orange"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v between "bee" and "orange"
+POSTHOOK: query: select * from newtypestbl where v between "bee" and "orange"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v between "bee" and "orange"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v between "bee" and "orange"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v between "bee" and "orange"
+POSTHOOK: query: select * from newtypestbl where v between "bee" and "orange"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v between "bee" and "zombie"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v between "bee" and "zombie"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v between "bee" and "zombie"
+POSTHOOK: query: select * from newtypestbl where v between "bee" and "zombie"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v between "bee" and "zombie"
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v between "bee" and "zombie"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v between "bee" and "zombie"
+POSTHOOK: query: select * from newtypestbl where v between "bee" and "zombie"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v between "orange" and "pine"
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where v between "orange" and "pine"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v between "orange" and "pine"
+POSTHOOK: query: select * from newtypestbl where v between "orange" and "pine"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where v between "orange" and "pine"
+PREHOOK: query: select * from newtypestbl where v between "orange" and "pine"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where v between "orange" and "pine"
+POSTHOOK: query: select * from newtypestbl where v between "orange" and "pine"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/results/clientpositive/parquet_predicate_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_predicate_pushdown.q.out b/ql/src/test/results/clientpositive/parquet_predicate_pushdown.q.out
index 1dc2937..aa3b272 100644
--- a/ql/src/test/results/clientpositive/parquet_predicate_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/parquet_predicate_pushdown.q.out
@@ -251,45 +251,65 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tbl_pred
 #### A masked pattern was here ####
 124	336	65664	4294967435	74.72	42.47	true	bob davidson	2013-03-01 09:11:58.703302	45.4	yard duty
-PREHOOK: query: SELECT SUM(HASH(t)) FROM tbl_pred
+PREHOOK: query: SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
   AND t > -2
+  LIMIT 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tbl_pred
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT SUM(HASH(t)) FROM tbl_pred
+POSTHOOK: query: SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
   AND t > -2
+  LIMIT 10
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tbl_pred
 #### A masked pattern was here ####
--8
-PREHOOK: query: SELECT SUM(HASH(t)) FROM tbl_pred
+-1	268	65778	4294967418	56.33	44.73	true	calvin falkner	2013-03-01 09:11:58.70322	7.37	history
+-1	281	65643	4294967323	15.1	45.0	false	irene nixon	2013-03-01 09:11:58.703223	80.96	undecided
+-1	300	65663	4294967343	71.26	34.62	true	calvin ovid	2013-03-01 09:11:58.703262	78.56	study skills
+-1	348	65556	4294967413	35.17	9.51	false	bob young	2013-03-01 09:11:58.70328	45.81	quiet hour
+-1	372	65680	4294967490	15.45	18.09	false	ethan laertes	2013-03-01 09:11:58.70311	65.88	opthamology
+-1	417	65685	4294967492	28.89	5.19	true	mike white	2013-03-01 09:11:58.703275	90.69	forestry
+-1	423	65663	4294967380	0.79	21.33	false	bob laertes	2013-03-01 09:11:58.703278	94.16	debate
+-1	433	65581	4294967299	86.92	23.15	false	yuri ellison	2013-03-01 09:11:58.703098	21.29	history
+PREHOOK: query: SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
   AND t > -2
+  LIMIT 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tbl_pred
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT SUM(HASH(t)) FROM tbl_pred
+POSTHOOK: query: SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
   AND t > -2
+  LIMIT 10
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tbl_pred
 #### A masked pattern was here ####
--8
-PREHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+-1	268	65778	4294967418	56.33	44.73	true	calvin falkner	2013-03-01 09:11:58.70322	7.37	history
+-1	281	65643	4294967323	15.1	45.0	false	irene nixon	2013-03-01 09:11:58.703223	80.96	undecided
+-1	300	65663	4294967343	71.26	34.62	true	calvin ovid	2013-03-01 09:11:58.703262	78.56	study skills
+-1	348	65556	4294967413	35.17	9.51	false	bob young	2013-03-01 09:11:58.70328	45.81	quiet hour
+-1	372	65680	4294967490	15.45	18.09	false	ethan laertes	2013-03-01 09:11:58.70311	65.88	opthamology
+-1	417	65685	4294967492	28.89	5.19	true	mike white	2013-03-01 09:11:58.703275	90.69	forestry
+-1	423	65663	4294967380	0.79	21.33	false	bob laertes	2013-03-01 09:11:58.703278	94.16	debate
+-1	433	65581	4294967299	86.92	23.15	false	yuri ellison	2013-03-01 09:11:58.703098	21.29	history
+PREHOOK: query: EXPLAIN SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
   AND t > -2
+  LIMIT 10
 PREHOOK: type: QUERY
-POSTHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+POSTHOOK: query: EXPLAIN SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
   AND t > -2
+  LIMIT 10
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
@@ -306,47 +326,37 @@ STAGE PLANS:
               predicate: ((t < 0) and (UDFToInteger(t) > -2)) (type: boolean)
               Statistics: Num rows: 116 Data size: 1276 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: hash(t) (type: int)
-                outputColumnNames: _col0
+                expressions: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                 Statistics: Num rows: 116 Data size: 1276 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: sum(_col0)
-                  mode: hash
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    sort order: 
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: bigint)
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: sum(VALUE._col0)
-          mode: mergepartial
-          outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            table:
-                input format: org.apache.hadoop.mapred.TextInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                Limit
+                  Number of rows: 10
+                  Statistics: Num rows: 10 Data size: 110 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 10 Data size: 110 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
-      limit: -1
+      limit: 10
       Processor Tree:
         ListSink
 
-PREHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+PREHOOK: query: EXPLAIN SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
   AND t > -2
+  LIMIT 10
 PREHOOK: type: QUERY
-POSTHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+POSTHOOK: query: EXPLAIN SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
   AND t > -2
+  LIMIT 10
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
@@ -364,35 +374,23 @@ STAGE PLANS:
               predicate: ((t < 0) and (UDFToInteger(t) > -2)) (type: boolean)
               Statistics: Num rows: 116 Data size: 1276 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: hash(t) (type: int)
-                outputColumnNames: _col0
+                expressions: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                 Statistics: Num rows: 116 Data size: 1276 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: sum(_col0)
-                  mode: hash
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    sort order: 
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: bigint)
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: sum(VALUE._col0)
-          mode: mergepartial
-          outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            table:
-                input format: org.apache.hadoop.mapred.TextInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                Limit
+                  Number of rows: 10
+                  Statistics: Num rows: 10 Data size: 110 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 10 Data size: 110 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
-      limit: -1
+      limit: 10
       Processor Tree:
         ListSink
 


[44/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/ql_rewrite_gbtoidx_cbo_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/ql_rewrite_gbtoidx_cbo_1.q.out b/ql/src/test/results/clientpositive/spark/ql_rewrite_gbtoidx_cbo_1.q.out
index 02e24c3..878d026 100644
--- a/ql/src/test/results/clientpositive/spark/ql_rewrite_gbtoidx_cbo_1.q.out
+++ b/ql/src/test/results/clientpositive/spark/ql_rewrite_gbtoidx_cbo_1.q.out
@@ -97,11 +97,11 @@ STAGE PLANS:
                   Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: l_shipdate (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: l_shipdate
                     Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
-                      keys: _col0 (type: string)
+                      aggregations: count(l_shipdate)
+                      keys: l_shipdate (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
@@ -264,15 +264,15 @@ STAGE PLANS:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: $hdt$_0:default__lineitem_ix_lineitem_ix_lshipdate_idx__
+                  alias: default__lineitem_ix_lineitem_ix_lshipdate_idx__
                   Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint)
-                    outputColumnNames: _col0, _count_of_l_shipdate
+                    outputColumnNames: l_shipdate, _count_of_l_shipdate
                     Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: sum(_count_of_l_shipdate)
-                      keys: _col0 (type: string)
+                      keys: l_shipdate (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
@@ -912,15 +912,15 @@ STAGE PLANS:
         Map 2 
             Map Operator Tree:
                 TableScan
-                  alias: null-subquery1:$hdt$_0-subquery1:$hdt$_0:default__lineitem_ix_lineitem_ix_lshipdate_idx__
+                  alias: null-subquery1:$hdt$_0-subquery1:default__lineitem_ix_lineitem_ix_lshipdate_idx__
                   Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint)
-                    outputColumnNames: _col0, _count_of_l_shipdate
+                    outputColumnNames: l_shipdate, _count_of_l_shipdate
                     Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: sum(_count_of_l_shipdate)
-                      keys: _col0 (type: string)
+                      keys: l_shipdate (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
@@ -1004,11 +1004,11 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: 1 (type: int)
-                      outputColumnNames: _col0
+                      outputColumnNames: key
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        aggregations: count(_col0)
-                        keys: _col0 (type: int)
+                        aggregations: count(key)
+                        keys: key (type: int)
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1058,15 +1058,15 @@ STAGE PLANS:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: $hdt$_0:default__tbl_tbl_key_idx__
+                  alias: default__tbl_tbl_key_idx__
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
                     expressions: key (type: int), _count_of_key (type: bigint)
-                    outputColumnNames: _col0, _count_of_key
+                    outputColumnNames: key, _count_of_key
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
                       aggregations: sum(_count_of_key)
-                      keys: _col0 (type: int)
+                      keys: key (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1168,11 +1168,11 @@ STAGE PLANS:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: $hdt$_0:default__tbl_tbl_key_idx__
+                  alias: default__tbl_tbl_key_idx__
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
                     expressions: key (type: int), _count_of_key (type: bigint)
-                    outputColumnNames: _col0, _count_of_key
+                    outputColumnNames: key, _count_of_key
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
                       aggregations: sum(_count_of_key)
@@ -1226,10 +1226,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
                     expressions: key (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int)
+                      keys: key (type: int)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1280,11 +1280,11 @@ STAGE PLANS:
                   alias: tbl
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
-                    expressions: value (type: int), key (type: int)
-                    outputColumnNames: _col0, _col1
+                    expressions: key (type: int), value (type: int)
+                    outputColumnNames: key, value
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int), _col1 (type: int)
+                      keys: key (type: int), value (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1301,7 +1301,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Select Operator
-                  expressions: _col1 (type: int)
+                  expressions: _col0 (type: int)
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   File Output Operator
@@ -1343,10 +1343,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: 3 (type: int)
-                      outputColumnNames: _col0
+                      outputColumnNames: key
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        keys: _col0 (type: int)
+                        keys: key (type: int)
                         mode: hash
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1401,10 +1401,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: key (type: int)
-                      outputColumnNames: _col0
+                      outputColumnNames: key
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        keys: _col0 (type: int)
+                        keys: key (type: int)
                         mode: hash
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1514,11 +1514,11 @@ STAGE PLANS:
                   alias: tbl
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
-                    expressions: value (type: int), key (type: int)
-                    outputColumnNames: _col0, _col1
+                    expressions: key (type: int), value (type: int)
+                    outputColumnNames: key, value
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int), _col1 (type: int)
+                      keys: key (type: int), value (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1534,17 +1534,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: int), _col0 (type: int)
-                  outputColumnNames: _col0, _col1
+                File Output Operator
+                  compressed: false
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.TextInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -1577,10 +1573,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: key (type: int), 1 (type: int)
-                      outputColumnNames: _col0, _col1
+                      outputColumnNames: key, value
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        keys: _col0 (type: int), _col1 (type: int)
+                        keys: key (type: int), value (type: int)
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1632,10 +1628,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
                     expressions: key (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int)
+                      keys: key (type: int)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1687,10 +1683,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
                     expressions: key (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int)
+                      keys: key (type: int)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1742,10 +1738,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
                     expressions: key (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int)
+                      keys: key (type: int)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1797,10 +1793,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
                     expressions: key (type: int), value (type: int)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int), _col1 (type: int)
+                      keys: key (type: int), value (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1855,10 +1851,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: key (type: int), 2 (type: int)
-                      outputColumnNames: _col0, _col1
+                      outputColumnNames: key, value
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        keys: _col0 (type: int), _col1 (type: int)
+                        keys: key (type: int), value (type: int)
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1913,10 +1909,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: 3 (type: int), 2 (type: int)
-                      outputColumnNames: _col0, _col1
+                      outputColumnNames: key, value
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        keys: _col0 (type: int), _col1 (type: int)
+                        keys: key (type: int), value (type: int)
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1969,20 +1965,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (value = key) (type: boolean)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                    Select Operator
-                      expressions: key (type: int), value (type: int)
+                    Group By Operator
+                      keys: key (type: int), value (type: int)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: int), _col1 (type: int)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int), _col1 (type: int)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int), _col1 (type: int)
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
-                          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
@@ -2142,10 +2134,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: key (type: int), 2 (type: int)
-                      outputColumnNames: _col0, _col1
+                      outputColumnNames: key, value
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        keys: _col0 (type: int), _col1 (type: int)
+                        keys: key (type: int), value (type: int)
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -2283,22 +2275,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (key < 10) (type: boolean)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
+                    Group By Operator
+                      aggregations: count(key)
+                      keys: key (type: int)
+                      mode: hash
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: count(_col0)
-                        keys: _col0 (type: int)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: bigint)
+                        value expressions: _col1 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
@@ -2381,18 +2369,18 @@ STAGE PLANS:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: $hdt$_0:default__tblpart_tbl_part_index__
+                  alias: default__tblpart_tbl_part_index__
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (key < 10) (type: boolean)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key (type: int), _count_of_key (type: bigint)
-                      outputColumnNames: _col0, _count_of_key
+                      outputColumnNames: key, _count_of_key
                       Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: sum(_count_of_key)
-                        keys: _col0 (type: int)
+                        keys: key (type: int)
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
@@ -2496,11 +2484,11 @@ STAGE PLANS:
                   Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
-                      keys: _col0 (type: int)
+                      aggregations: count(key)
+                      keys: key (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE
@@ -2576,15 +2564,15 @@ STAGE PLANS:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: $hdt$_0:default__tbl_tbl_key_idx__
+                  alias: default__tbl_tbl_key_idx__
                   Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: int), _count_of_key (type: bigint)
-                    outputColumnNames: _col0, _count_of_key
+                    outputColumnNames: key, _count_of_key
                     Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: sum(_count_of_key)
-                      keys: _col0 (type: int)
+                      keys: key (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/stats_only_null.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats_only_null.q.out b/ql/src/test/results/clientpositive/spark/stats_only_null.q.out
index cb0920e..c88e53b 100644
--- a/ql/src/test/results/clientpositive/spark/stats_only_null.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats_only_null.q.out
@@ -90,10 +90,10 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: a (type: double), b (type: int), c (type: string), d (type: smallint)
-                    outputColumnNames: _col0, _col1, _col2, _col3
+                    outputColumnNames: a, b, c, d
                     Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(), count(_col0), count(_col1), count(_col2), count(_col3)
+                      aggregations: count(), count(a), count(b), count(c), count(d)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
@@ -146,10 +146,10 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: a (type: double), b (type: int), c (type: string), d (type: smallint)
-                    outputColumnNames: _col0, _col1, _col2, _col3
+                    outputColumnNames: a, b, c, d
                     Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(), count(_col0), count(_col1), count(_col2), count(_col3)
+                      aggregations: count(), count(a), count(b), count(c), count(d)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/subquery_in.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/subquery_in.q.out b/ql/src/test/results/clientpositive/spark/subquery_in.q.out
index bfcdaa8..b2a1681 100644
--- a/ql/src/test/results/clientpositive/spark/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/spark/subquery_in.q.out
@@ -646,20 +646,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: ((key > '9') and value is not null) (type: boolean)
                     Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      keys: key (type: string), value (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: string), _col1 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: string)
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                          Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Join Operator
@@ -829,20 +825,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: l_partkey is not null (type: boolean)
                     Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: l_partkey (type: int)
+                    Group By Operator
+                      keys: l_partkey (type: int)
+                      mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: int)
-                        mode: hash
-                        outputColumnNames: _col0
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Join Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/union11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union11.q.out b/ql/src/test/results/clientpositive/spark/union11.q.out
index a6b9367..ab7222e 100644
--- a/ql/src/test/results/clientpositive/spark/union11.q.out
+++ b/ql/src/test/results/clientpositive/spark/union11.q.out
@@ -36,17 +36,19 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: key (type: string)
+                    outputColumnNames: key
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(1)
+                      aggregations: count(key)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         sort order: 
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
@@ -54,22 +56,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst1' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
         Reducer 3 
             Reduce Operator Tree:
@@ -78,10 +80,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -92,22 +94,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst2' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
         Reducer 7 
             Reduce Operator Tree:
@@ -115,22 +117,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst3' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
 
   Stage: Stage-0
@@ -155,6 +157,6 @@ POSTHOOK: query: select unionsrc.key, count(1) FROM (select 'tst1' as key, count
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 #### A masked pattern was here ####
-tst2	1
 tst3	1
 tst1	1
+tst2	1

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/union14.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union14.q.out b/ql/src/test/results/clientpositive/spark/union14.q.out
index 0c9542b..f276d7f 100644
--- a/ql/src/test/results/clientpositive/spark/union14.q.out
+++ b/ql/src/test/results/clientpositive/spark/union14.q.out
@@ -42,28 +42,30 @@ STAGE PLANS:
                       keys: _col0 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: bigint)
         Map 3 
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: key (type: string)
+                    outputColumnNames: key
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(1)
+                      aggregations: count(key)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         sort order: 
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
@@ -72,10 +74,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -86,22 +88,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst1' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
 
   Stage: Stage-0

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/union15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union15.q.out b/ql/src/test/results/clientpositive/spark/union15.q.out
index cb8bc75..fc1eb83 100644
--- a/ql/src/test/results/clientpositive/spark/union15.q.out
+++ b/ql/src/test/results/clientpositive/spark/union15.q.out
@@ -34,17 +34,19 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: key (type: string)
+                    outputColumnNames: key
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(1)
+                      aggregations: count(key)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         sort order: 
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
         Map 4 
             Map Operator Tree:
@@ -60,12 +62,12 @@ STAGE PLANS:
                       keys: _col0 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
@@ -73,22 +75,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst1' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 51 Data size: 390 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
         Reducer 3 
             Reduce Operator Tree:
@@ -97,10 +99,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/union28.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union28.q.out b/ql/src/test/results/clientpositive/spark/union28.q.out
index 59657cc..7ee06fe 100644
--- a/ql/src/test/results/clientpositive/spark/union28.q.out
+++ b/ql/src/test/results/clientpositive/spark/union28.q.out
@@ -73,10 +73,10 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string), _col1 (type: string)
+                      keys: key (type: string), value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/union30.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union30.q.out b/ql/src/test/results/clientpositive/spark/union30.q.out
index fd907f9..12eda1d 100644
--- a/ql/src/test/results/clientpositive/spark/union30.q.out
+++ b/ql/src/test/results/clientpositive/spark/union30.q.out
@@ -87,10 +87,10 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string), _col1 (type: string)
+                      keys: key (type: string), value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/union33.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union33.q.out b/ql/src/test/results/clientpositive/spark/union33.q.out
index 271c377..25e545c 100644
--- a/ql/src/test/results/clientpositive/spark/union33.q.out
+++ b/ql/src/test/results/clientpositive/spark/union33.q.out
@@ -72,11 +72,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: string)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -205,11 +205,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: string)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/union5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union5.q.out b/ql/src/test/results/clientpositive/spark/union5.q.out
index 06a5d90..af0528a 100644
--- a/ql/src/test/results/clientpositive/spark/union5.q.out
+++ b/ql/src/test/results/clientpositive/spark/union5.q.out
@@ -31,17 +31,19 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: key (type: string)
+                    outputColumnNames: key
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(1)
+                      aggregations: count(key)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         sort order: 
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
@@ -49,22 +51,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst1' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
         Reducer 3 
             Reduce Operator Tree:
@@ -73,10 +75,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -87,22 +89,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst2' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
 
   Stage: Stage-0
@@ -123,5 +125,5 @@ POSTHOOK: query: select unionsrc.key, count(1) FROM (select 'tst1' as key, count
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 #### A masked pattern was here ####
-tst2	1
 tst1	1
+tst2	1

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/union7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union7.q.out b/ql/src/test/results/clientpositive/spark/union7.q.out
index 4a81283..181d04c 100644
--- a/ql/src/test/results/clientpositive/spark/union7.q.out
+++ b/ql/src/test/results/clientpositive/spark/union7.q.out
@@ -30,17 +30,19 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: key (type: string)
+                    outputColumnNames: key
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(1)
+                      aggregations: count(key)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         sort order: 
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
         Map 4 
             Map Operator Tree:
@@ -56,12 +58,12 @@ STAGE PLANS:
                       keys: _col0 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
@@ -69,22 +71,22 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: 'tst1' (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     keys: _col0 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 26 Data size: 199 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
         Reducer 3 
             Reduce Operator Tree:
@@ -93,10 +95,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/union_remove_21.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_21.q.out b/ql/src/test/results/clientpositive/spark/union_remove_21.q.out
index 246594f..a4af758 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_21.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_21.q.out
@@ -81,10 +81,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out b/ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out
index ee5462d..c8df568 100644
--- a/ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out
@@ -1258,10 +1258,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ws_order_number (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: ws_order_number
                     Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int)
+                      keys: ws_order_number (type: int)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out b/ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out
index 7e161d1..cb2d56b 100644
--- a/ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out
@@ -54,11 +54,11 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cint (type: int), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14))
-                    outputColumnNames: _col0, _col1, _col2
+                    outputColumnNames: cint, cdecimal1, cdecimal2
                     Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1), max(_col1), min(_col1), sum(_col1), count(_col2), max(_col2), min(_col2), sum(_col2), count()
-                      keys: _col0 (type: int)
+                      aggregations: count(cdecimal1), max(cdecimal1), min(cdecimal1), sum(cdecimal1), count(cdecimal2), max(cdecimal2), min(cdecimal2), sum(cdecimal2), count()
+                      keys: cint (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                       Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
@@ -159,11 +159,11 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cint (type: int), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14))
-                    outputColumnNames: _col0, _col1, _col2
+                    outputColumnNames: cint, cdecimal1, cdecimal2
                     Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1), max(_col1), min(_col1), sum(_col1), avg(_col1), stddev_pop(_col1), stddev_samp(_col1), count(_col2), max(_col2), min(_col2), sum(_col2), avg(_col2), stddev_pop(_col2), stddev_samp(_col2), count()
-                      keys: _col0 (type: int)
+                      aggregations: count(cdecimal1), max(cdecimal1), min(cdecimal1), sum(cdecimal1), avg(cdecimal1), stddev_pop(cdecimal1), stddev_samp(cdecimal1), count(cdecimal2), max(cdecimal2), min(cdecimal2), sum(cdecimal2), avg(cdecimal2), stddev_pop(cdecimal2), stddev_samp(cdecimal2), count()
+                      keys: cint (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15
                       Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out b/ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out
index 322270f..52c00f9 100644
--- a/ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out
@@ -128,34 +128,38 @@ STAGE PLANS:
                   alias: vectortab2korc
                   Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: s (type: string), t (type: tinyint)
-                    outputColumnNames: _col0, _col1
+                    expressions: t (type: tinyint), s (type: string)
+                    outputColumnNames: t, s
                     Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string), _col1 (type: tinyint)
+                      keys: t (type: tinyint), s (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: string), _col1 (type: tinyint)
+                        key expressions: _col0 (type: tinyint), _col1 (type: string)
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
+                        Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: string)
                         Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
+                Select Operator
+                  expressions: _col1 (type: string), _col0 (type: tinyint)
+                  outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
             Execution mode: vectorized
 
   Stage: Stage-0

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out b/ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out
index 3d6a236..2255f72 100644
--- a/ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out
@@ -128,19 +128,19 @@ STAGE PLANS:
                   alias: vectortab2korc
                   Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: s (type: string), t (type: tinyint), b (type: bigint)
-                    outputColumnNames: _col0, _col1, _col2
+                    expressions: t (type: tinyint), s (type: string), b (type: bigint)
+                    outputColumnNames: t, s, b
                     Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col2)
-                      keys: _col0 (type: string), _col1 (type: tinyint)
+                      aggregations: max(b)
+                      keys: t (type: tinyint), s (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: string), _col1 (type: tinyint)
+                        key expressions: _col0 (type: tinyint), _col1 (type: string)
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
+                        Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: string)
                         Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col2 (type: bigint)
             Execution mode: vectorized
@@ -148,17 +148,21 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: max(VALUE._col0)
-                keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
+                Select Operator
+                  expressions: _col1 (type: string), _col0 (type: tinyint), _col2 (type: bigint)
+                  outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
             Execution mode: vectorized
 
   Stage: Stage-0

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out b/ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out
index be39d0d..bbc66fc 100644
--- a/ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out
@@ -65,20 +65,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: l_partkey is not null (type: boolean)
                     Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: l_partkey (type: int)
+                    Group By Operator
+                      keys: l_partkey (type: int)
+                      mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: int)
-                        mode: hash
-                        outputColumnNames: _col0
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
         Reducer 4 
             Local Work:
               Map Reduce Local Work
@@ -270,20 +266,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: l_partkey is not null (type: boolean)
                     Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: l_partkey (type: int)
+                    Group By Operator
+                      keys: l_partkey (type: int)
+                      mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: int)
-                        mode: hash
-                        outputColumnNames: _col0
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
         Reducer 4 
             Local Work:
               Map Reduce Local Work

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out b/ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out
index 6bbb6b9..3363c8b 100644
--- a/ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out
@@ -126,11 +126,11 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: bo (type: boolean), b (type: bigint)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: bo, b
                     Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col1)
-                      keys: _col0 (type: boolean)
+                      aggregations: max(b)
+                      keys: bo (type: boolean)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vectorization_0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorization_0.q.out b/ql/src/test/results/clientpositive/spark/vectorization_0.q.out
index 3ad059c..ba33bfb 100644
--- a/ql/src/test/results/clientpositive/spark/vectorization_0.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorization_0.q.out
@@ -133,10 +133,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctinyint
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(ctinyint)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -439,10 +439,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cbigint (type: bigint)
-                    outputColumnNames: _col0
+                    outputColumnNames: cbigint
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(cbigint)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -745,10 +745,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cfloat (type: float)
-                    outputColumnNames: _col0
+                    outputColumnNames: cfloat
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(cfloat)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -998,10 +998,10 @@ STAGE PLANS:
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cbigint (type: bigint), cfloat (type: float), ctinyint (type: tinyint)
-                      outputColumnNames: _col0, _col1, _col2
+                      outputColumnNames: cbigint, cfloat, ctinyint
                       Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: avg(_col0), stddev_pop(_col0), var_samp(_col0), count(), sum(_col1), min(_col2)
+                        aggregations: avg(cbigint), stddev_pop(cbigint), var_samp(cbigint), count(), sum(cfloat), min(ctinyint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                         Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE


[28/50] [abbrv] hive git commit: HIVE-11814: Emit query time in lineage info (Jimmy, reviewed by Ashutosh)

Posted by xu...@apache.org.
HIVE-11814: Emit query time in lineage info (Jimmy, reviewed by Ashutosh)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/07ca8120
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/07ca8120
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/07ca8120

Branch: refs/heads/spark
Commit: 07ca81202c9b0138d562fa863cbd60073789bf12
Parents: cabd481
Author: Jimmy Xiang <jx...@cloudera.com>
Authored: Mon Sep 14 09:25:04 2015 -0700
Committer: Jimmy Xiang <jx...@cloudera.com>
Committed: Tue Sep 15 09:14:38 2015 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/07ca8120/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
index 3c6ce94..f615d81 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
@@ -149,8 +149,10 @@ public class LineageLogger implements ExecuteWithHookContext {
           // so that the test golden output file is fixed.
           long queryTime = plan.getQueryStartTime().longValue();
           if (queryTime == 0) queryTime = System.currentTimeMillis();
+          long duration = System.currentTimeMillis() - queryTime;
           writer.name("user").value(hookContext.getUgi().getUserName());
           writer.name("timestamp").value(queryTime/1000);
+          writer.name("duration").value(duration);
           writer.name("jobIds");
           writer.beginArray();
           List<TaskRunner> tasks = hookContext.getCompleteTaskList();


[33/50] [abbrv] hive git commit: HIVE-11824: Insert to local directory causes staging directory to be copied (Prasanth Jayachandran reviewed by Ashutosh Chauhan)

Posted by xu...@apache.org.
HIVE-11824: Insert to local directory causes staging directory to be copied (Prasanth Jayachandran reviewed by Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/03f46b2f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/03f46b2f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/03f46b2f

Branch: refs/heads/spark
Commit: 03f46b2f2eb0235e9a2079079e6d643e8cf7c89e
Parents: ba0b33c
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Wed Sep 16 01:08:02 2015 -0500
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Wed Sep 16 01:08:02 2015 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/exec/MoveTask.java    | 16 ++++++++++++++--
 1 file changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/03f46b2f/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index a1f8973..6a19cc3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -127,10 +127,22 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
       LocalFileSystem dstFs = FileSystem.getLocal(conf);
 
       if (dstFs.delete(targetPath, true) || !dstFs.exists(targetPath)) {
-        console.printInfo(mesg, mesg_detail);
         // if source exists, rename. Otherwise, create a empty directory
         if (fs.exists(sourcePath)) {
-          fs.copyToLocalFile(sourcePath, targetPath);
+          try {
+            // create the destination if it does not exist
+            if (!dstFs.exists(targetPath)) {
+              FileUtils.mkdir(dstFs, targetPath, false, conf);
+            }
+          } catch (IOException e) {
+            throw new HiveException("Unable to create target directory for copy" + targetPath, e);
+          }
+
+          FileSystem srcFs = sourcePath.getFileSystem(conf);
+          FileStatus[] srcs = srcFs.listStatus(sourcePath, FileUtils.HIDDEN_FILES_PATH_FILTER);
+          for (FileStatus status : srcs) {
+            fs.copyToLocalFile(status.getPath(), targetPath);
+          }
         } else {
           if (!dstFs.mkdirs(targetPath)) {
             throw new HiveException("Unable to make local directory: "


[50/50] [abbrv] hive git commit: HIVE-11844: Merge master to Spark branch 9/16/2015 [Spark Branch]

Posted by xu...@apache.org.
HIVE-11844: Merge master to Spark branch 9/16/2015 [Spark Branch]


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/70eeadd2
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/70eeadd2
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/70eeadd2

Branch: refs/heads/spark
Commit: 70eeadd2f019dcb2e301690290c8807731eab7a1
Parents: f78f663 1cce5f0
Author: xzhang <xz...@xzdt>
Authored: Wed Sep 16 12:16:30 2015 -0700
Committer: xzhang <xz...@xzdt>
Committed: Wed Sep 16 12:16:30 2015 -0700

----------------------------------------------------------------------
 accumulo-handler/pom.xml                        |     4 -
 .../apache/hadoop/hive/ant/GenVectorCode.java   |   105 +
 .../java/org/apache/hive/beeline/BeeLine.java   |    13 +-
 .../org/apache/hive/beeline/BeeLineOpts.java    |    12 +-
 .../org/apache/hive/beeline/HiveSchemaTool.java |    14 +-
 .../src/main/resources/beeline-log4j.properties |    24 -
 beeline/src/main/resources/beeline-log4j2.xml   |    39 +
 bin/ext/beeline.sh                              |     2 +-
 bin/hive                                        |     3 +
 common/pom.xml                                  |    22 +-
 .../apache/hadoop/hive/common/JavaUtils.java    |    11 +-
 .../org/apache/hadoop/hive/common/LogUtils.java |    18 +-
 .../hadoop/hive/common/jsonexplain/tez/Op.java  |     8 +-
 .../hive/common/jsonexplain/tez/Stage.java      |    14 +-
 .../common/jsonexplain/tez/TezJsonParser.java   |    17 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   120 +-
 common/src/main/resources/hive-log4j.properties |    88 -
 common/src/main/resources/hive-log4j2.xml       |   110 +
 .../hadoop/hive/conf/TestHiveLogging.java       |     8 +-
 .../resources/hive-exec-log4j-test.properties   |    59 -
 .../test/resources/hive-exec-log4j2-test.xml    |    85 +
 .../test/resources/hive-log4j-test.properties   |    71 -
 common/src/test/resources/hive-log4j2-test.xml  |    94 +
 data/conf/hive-log4j-old.properties             |    82 -
 data/conf/hive-log4j.properties                 |    97 -
 data/conf/hive-log4j2.xml                       |   143 +
 data/conf/hive-site.xml                         |     6 -
 data/conf/spark/log4j.properties                |    24 -
 data/conf/spark/log4j2.xml                      |    74 +
 data/conf/tez/hive-site.xml                     |     9 +
 data/files/dynpartdata1.txt                     |     5 +
 data/files/dynpartdata2.txt                     |     6 +
 docs/xdocs/language_manual/cli.xml              |     2 +-
 errata.txt                                      |    10 +
 .../hadoop/hive/hbase/ColumnMappings.java       |     5 +
 .../apache/hadoop/hive/hbase/HBaseSerDe.java    |    19 +-
 .../hadoop/hive/hbase/HBaseSerDeParameters.java |     8 +-
 .../hadoop/hive/hbase/HBaseStorageHandler.java  |    13 +-
 .../hive/hbase/HiveHBaseInputFormatUtil.java    |    50 +-
 .../hadoop/hive/hbase/LazyHBaseCellMap.java     |    19 +-
 .../apache/hadoop/hive/hbase/LazyHBaseRow.java  |     5 +-
 .../hadoop/hive/hbase/LazyHBaseCellMapTest.java |    72 +
 .../positive/hbase_binary_map_queries_prefix.q  |    15 +-
 .../queries/positive/hbase_null_first_col.q     |    22 +
 .../hbase_binary_map_queries_prefix.q.out       |    40 +
 .../results/positive/hbase_null_first_col.q.out |   109 +
 .../test/results/positive/hbase_timestamp.q.out |     8 +-
 hcatalog/bin/hcat_server.sh                     |     2 +-
 hcatalog/bin/templeton.cmd                      |     4 +-
 .../mapreduce/DefaultOutputFormatContainer.java |     7 +-
 ...namicPartitionFileRecordWriterContainer.java |     3 +-
 .../mapreduce/FileOutputFormatContainer.java    |     3 +-
 hcatalog/scripts/hcat_server_start.sh           |     2 +-
 .../content/xdocs/configuration.xml             |     2 +-
 .../src/documentation/content/xdocs/install.xml |     2 +-
 .../deployers/config/hive/hive-log4j.properties |    88 -
 .../deployers/config/hive/hive-log4j2.xml       |   110 +
 .../deployers/config/hive/hive-site.mysql.xml   |    22 +
 .../templeton/deployers/start_hive_services.sh  |     2 +-
 .../hive/hcatalog/streaming/TestStreaming.java  |    54 +-
 .../webhcat/svr/src/main/bin/webhcat_server.sh  |     4 +-
 .../src/main/config/webhcat-log4j.properties    |    45 -
 .../svr/src/main/config/webhcat-log4j2.xml      |    74 +
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     |   201 +-
 .../java/org/apache/hive/hplsql/Column.java     |    65 +
 .../java/org/apache/hive/hplsql/Converter.java  |    41 +-
 .../main/java/org/apache/hive/hplsql/Exec.java  |   184 +-
 .../java/org/apache/hive/hplsql/Expression.java |    51 +-
 .../main/java/org/apache/hive/hplsql/Meta.java  |   216 +
 .../main/java/org/apache/hive/hplsql/Row.java   |    97 +
 .../java/org/apache/hive/hplsql/Select.java     |    63 +-
 .../java/org/apache/hive/hplsql/Signal.java     |     2 +-
 .../main/java/org/apache/hive/hplsql/Stmt.java  |   110 +-
 .../main/java/org/apache/hive/hplsql/Var.java   |    60 +-
 .../apache/hive/hplsql/functions/Function.java  |    62 +-
 .../hive/hplsql/functions/FunctionDatetime.java |    14 +-
 .../hive/hplsql/functions/FunctionMisc.java     |    22 +-
 .../hive/hplsql/functions/FunctionOra.java      |    31 +-
 .../hive/hplsql/functions/FunctionString.java   |    46 +-
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |    12 +-
 .../apache/hive/hplsql/TestHplsqlOffline.java   |    76 +
 .../test/queries/db/create_procedure_mssql.sql  |    52 +
 hplsql/src/test/queries/db/map_object.sql       |     9 +
 .../src/test/queries/db/rowtype_attribute.sql   |    22 +
 hplsql/src/test/queries/db/type_attribute.sql   |     8 +
 .../local/create_procedure_no_params.sql        |    19 +
 .../queries/local/exception_divide_by_zero.sql  |    11 +
 .../test/queries/offline/create_table_mssql.sql |    43 +
 .../test/queries/offline/create_table_ora.sql   |    53 +
 .../results/db/create_procedure_mssql.out.txt   |    45 +
 hplsql/src/test/results/db/map_object.out.txt   |    17 +
 .../test/results/db/rowtype_attribute.out.txt   |    42 +
 .../src/test/results/db/type_attribute.out.txt  |    15 +
 .../test/results/local/create_function.out.txt  |     4 +-
 .../local/create_procedure_no_params.out.txt    |    26 +
 hplsql/src/test/results/local/declare.out.txt   |     4 +-
 .../local/exception_divide_by_zero.out.txt      |     8 +
 .../results/offline/create_table_mssql.out.txt  |    24 +
 .../results/offline/create_table_ora.out.txt    |    42 +
 .../benchmark/serde/LazySimpleSerDeBench.java   |   453 +
 .../vectorization/VectorizationBench.java       |    93 +
 .../hive/metastore/TestHiveMetaStore.java       |    96 +-
 .../hive/ql/security/FolderPermissionBase.java  |    17 +-
 .../TestOperationLoggingAPIWithMr.java          |     2 -
 .../TestOperationLoggingAPIWithTez.java         |     2 -
 .../operation/TestOperationLoggingLayout.java   |   134 +
 itests/pom.xml                                  |     2 +-
 itests/qtest/pom.xml                            |    26 +-
 .../test/resources/testconfiguration.properties |     2 +
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |    62 +-
 jdbc/pom.xml                                    |     1 +
 .../org/apache/hive/jdbc/HiveConnection.java    |    30 +-
 .../org/apache/hive/jdbc/HiveStatement.java     |     2 +-
 jdbc/src/java/org/apache/hive/jdbc/Utils.java   |   117 +-
 .../hive/jdbc/ZooKeeperHiveClientHelper.java    |   104 +-
 metastore/if/hive_metastore.thrift              |     5 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  |  2046 ++-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |   135 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |     5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |   304 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |    47 +
 .../hive/metastore/api/AbortTxnRequest.java     |     2 +-
 .../metastore/api/AddDynamicPartitions.java     |     2 +-
 .../metastore/api/AddPartitionsRequest.java     |     2 +-
 .../hive/metastore/api/AddPartitionsResult.java |     2 +-
 .../hadoop/hive/metastore/api/AggrStats.java    |     2 +-
 .../metastore/api/AlreadyExistsException.java   |     2 +-
 .../metastore/api/BinaryColumnStatsData.java    |     2 +-
 .../metastore/api/BooleanColumnStatsData.java   |     2 +-
 .../hive/metastore/api/CheckLockRequest.java    |     2 +-
 .../hive/metastore/api/ColumnStatistics.java    |     2 +-
 .../metastore/api/ColumnStatisticsDesc.java     |     2 +-
 .../hive/metastore/api/ColumnStatisticsObj.java |     2 +-
 .../hive/metastore/api/CommitTxnRequest.java    |     2 +-
 .../hive/metastore/api/CompactionRequest.java   |     2 +-
 .../api/ConfigValSecurityException.java         |     2 +-
 .../api/CurrentNotificationEventId.java         |     2 +-
 .../hadoop/hive/metastore/api/Database.java     |     2 +-
 .../apache/hadoop/hive/metastore/api/Date.java  |     2 +-
 .../hive/metastore/api/DateColumnStatsData.java |     2 +-
 .../hadoop/hive/metastore/api/Decimal.java      |     2 +-
 .../metastore/api/DecimalColumnStatsData.java   |     2 +-
 .../metastore/api/DoubleColumnStatsData.java    |     2 +-
 .../hive/metastore/api/DropPartitionsExpr.java  |     2 +-
 .../metastore/api/DropPartitionsRequest.java    |     2 +-
 .../metastore/api/DropPartitionsResult.java     |     2 +-
 .../hive/metastore/api/EnvironmentContext.java  |     2 +-
 .../hadoop/hive/metastore/api/FieldSchema.java  |     2 +-
 .../hive/metastore/api/FireEventRequest.java    |     2 +-
 .../hive/metastore/api/FireEventResponse.java   |     2 +-
 .../hadoop/hive/metastore/api/Function.java     |     2 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   447 +
 .../metastore/api/GetOpenTxnsInfoResponse.java  |     2 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |     2 +-
 .../api/GetPrincipalsInRoleRequest.java         |     2 +-
 .../api/GetPrincipalsInRoleResponse.java        |     2 +-
 .../api/GetRoleGrantsForPrincipalRequest.java   |     2 +-
 .../api/GetRoleGrantsForPrincipalResponse.java  |     2 +-
 .../api/GrantRevokePrivilegeRequest.java        |     2 +-
 .../api/GrantRevokePrivilegeResponse.java       |     2 +-
 .../metastore/api/GrantRevokeRoleRequest.java   |     2 +-
 .../metastore/api/GrantRevokeRoleResponse.java  |     2 +-
 .../hive/metastore/api/HeartbeatRequest.java    |     2 +-
 .../metastore/api/HeartbeatTxnRangeRequest.java |     2 +-
 .../api/HeartbeatTxnRangeResponse.java          |     2 +-
 .../hive/metastore/api/HiveObjectPrivilege.java |     2 +-
 .../hive/metastore/api/HiveObjectRef.java       |     2 +-
 .../apache/hadoop/hive/metastore/api/Index.java |     2 +-
 .../api/IndexAlreadyExistsException.java        |     2 +-
 .../metastore/api/InsertEventRequestData.java   |     2 +-
 .../metastore/api/InvalidInputException.java    |     2 +-
 .../metastore/api/InvalidObjectException.java   |     2 +-
 .../api/InvalidOperationException.java          |     2 +-
 .../api/InvalidPartitionException.java          |     2 +-
 .../hive/metastore/api/LockComponent.java       |     2 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |     2 +-
 .../hadoop/hive/metastore/api/LockResponse.java |     2 +-
 .../hive/metastore/api/LongColumnStatsData.java |     2 +-
 .../hive/metastore/api/MetaException.java       |     2 +-
 .../hive/metastore/api/NoSuchLockException.java |     2 +-
 .../metastore/api/NoSuchObjectException.java    |     2 +-
 .../hive/metastore/api/NoSuchTxnException.java  |     2 +-
 .../hive/metastore/api/NotificationEvent.java   |     2 +-
 .../metastore/api/NotificationEventRequest.java |     2 +-
 .../api/NotificationEventResponse.java          |     2 +-
 .../hive/metastore/api/OpenTxnRequest.java      |     2 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Order.java |     2 +-
 .../hadoop/hive/metastore/api/Partition.java    |     2 +-
 .../api/PartitionListComposingSpec.java         |     2 +-
 .../hive/metastore/api/PartitionSpec.java       |     2 +-
 .../api/PartitionSpecWithSharedSD.java          |     2 +-
 .../hive/metastore/api/PartitionWithoutSD.java  |     2 +-
 .../metastore/api/PartitionsByExprRequest.java  |     2 +-
 .../metastore/api/PartitionsByExprResult.java   |     2 +-
 .../metastore/api/PartitionsStatsRequest.java   |     2 +-
 .../metastore/api/PartitionsStatsResult.java    |     2 +-
 .../metastore/api/PrincipalPrivilegeSet.java    |     2 +-
 .../hadoop/hive/metastore/api/PrivilegeBag.java |     2 +-
 .../hive/metastore/api/PrivilegeGrantInfo.java  |     2 +-
 .../hadoop/hive/metastore/api/ResourceUri.java  |     2 +-
 .../apache/hadoop/hive/metastore/api/Role.java  |     2 +-
 .../hive/metastore/api/RolePrincipalGrant.java  |     2 +-
 .../hadoop/hive/metastore/api/Schema.java       |     2 +-
 .../hadoop/hive/metastore/api/SerDeInfo.java    |     2 +-
 .../api/SetPartitionsStatsRequest.java          |     2 +-
 .../hive/metastore/api/ShowCompactRequest.java  |     2 +-
 .../hive/metastore/api/ShowCompactResponse.java |     2 +-
 .../api/ShowCompactResponseElement.java         |     2 +-
 .../hive/metastore/api/ShowLocksRequest.java    |     2 +-
 .../hive/metastore/api/ShowLocksResponse.java   |     2 +-
 .../metastore/api/ShowLocksResponseElement.java |     2 +-
 .../hadoop/hive/metastore/api/SkewedInfo.java   |     2 +-
 .../hive/metastore/api/StorageDescriptor.java   |     2 +-
 .../metastore/api/StringColumnStatsData.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Table.java |     2 +-
 .../hive/metastore/api/TableStatsRequest.java   |     2 +-
 .../hive/metastore/api/TableStatsResult.java    |     2 +-
 .../hive/metastore/api/ThriftHiveMetastore.java |  3140 ++--
 .../hive/metastore/api/TxnAbortedException.java |     2 +-
 .../hadoop/hive/metastore/api/TxnInfo.java      |     2 +-
 .../hive/metastore/api/TxnOpenException.java    |     2 +-
 .../apache/hadoop/hive/metastore/api/Type.java  |     2 +-
 .../hive/metastore/api/UnknownDBException.java  |     2 +-
 .../api/UnknownPartitionException.java          |     2 +-
 .../metastore/api/UnknownTableException.java    |     2 +-
 .../hive/metastore/api/UnlockRequest.java       |     2 +-
 .../hadoop/hive/metastore/api/Version.java      |     2 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   |  1265 +-
 .../src/gen/thrift/gen-php/metastore/Types.php  |   103 +
 .../hive_metastore/ThriftHiveMetastore-remote   |    21 +-
 .../hive_metastore/ThriftHiveMetastore.py       |   879 +-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |    74 +
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    16 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |    60 +
 .../hadoop/hive/metastore/HiveAlterHandler.java |     2 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |    53 +-
 .../hive/metastore/HiveMetaStoreClient.java     |     7 +
 .../hive/metastore/HouseKeeperService.java      |    39 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |     4 +
 .../hive/metastore/MetaStoreDirectSql.java      |    69 +-
 .../hadoop/hive/metastore/MetaStoreUtils.java   |    17 +-
 .../hadoop/hive/metastore/ObjectStore.java      |    54 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |     7 +
 .../hive/metastore/RetryingMetaStoreClient.java |    32 +-
 .../hive/metastore/tools/HiveMetaTool.java      |     5 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   178 +-
 .../DummyRawStoreControlledCommit.java          |     7 +
 .../DummyRawStoreForJdoConnection.java          |     6 +
 .../metastore/txn/TestCompactionTxnHandler.java |    40 +-
 .../hive/metastore/txn/TestTxnHandler.java      |    73 +-
 packaging/src/main/assembly/bin.xml             |    17 +-
 pom.xml                                         |    41 +-
 ql/pom.xml                                      |    17 +-
 .../hadoop/hive/ql/plan/api/Adjacency.java      |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Graph.java   |     2 +-
 .../hadoop/hive/ql/plan/api/Operator.java       |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Query.java   |     2 +-
 .../hadoop/hive/ql/plan/api/QueryPlan.java      |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Stage.java   |     2 +-
 .../apache/hadoop/hive/ql/plan/api/Task.java    |     2 +-
 ...tringGroupColumnCompareStringGroupColumn.txt |   112 +-
 ...gGroupColumnCompareStringGroupScalarBase.txt |    12 +-
 ...gGroupScalarCompareStringGroupColumnBase.txt |    12 +-
 ...tringGroupColumnCompareStringGroupColumn.txt |   112 +-
 ...gGroupColumnCompareStringGroupScalarBase.txt |    12 +-
 ...gGroupScalarCompareStringGroupColumnBase.txt |    12 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |    30 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |    11 +-
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |    24 +-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |     2 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    66 +-
 .../hadoop/hive/ql/exec/FunctionTask.java       |     2 +-
 .../hadoop/hive/ql/exec/KeyWrapperFactory.java  |    22 +-
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |     5 +
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |    30 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    |    28 +-
 .../hadoop/hive/ql/exec/ScriptOperator.java     |    85 +-
 .../apache/hadoop/hive/ql/exec/StatsTask.java   |    13 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |    26 +-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |    30 +-
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    |    20 +-
 .../persistence/BytesBytesMultiHashMap.java     |    11 +-
 .../persistence/HybridHashTableContainer.java   |    70 +-
 .../ql/exec/spark/RemoteHiveSparkClient.java    |    13 +-
 .../spark/status/impl/LocalSparkJobStatus.java  |     2 +-
 .../spark/status/impl/RemoteSparkJobStatus.java |     2 +-
 .../hadoop/hive/ql/exec/tez/DagUtils.java       |     3 +
 .../hive/ql/exec/tez/HashTableLoader.java       |     7 +-
 .../hadoop/hive/ql/exec/tez/InPlaceUpdates.java |    65 +
 .../hadoop/hive/ql/exec/tez/TezJobMonitor.java  |    66 +-
 .../hive/ql/exec/tez/TezSessionState.java       |     2 +
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |     4 +-
 .../ql/exec/vector/VectorHashKeyWrapper.java    |     2 +-
 .../ql/exec/vector/VectorizationContext.java    |    56 +-
 .../BRoundWithNumDigitsDoubleToDouble.java      |    42 +
 .../expressions/CastStringGroupToString.java    |    40 +
 .../ql/exec/vector/expressions/ColAndCol.java   |    34 +-
 .../ql/exec/vector/expressions/ColOrCol.java    |    42 +-
 .../exec/vector/expressions/CuckooSetBytes.java |     2 +-
 .../ql/exec/vector/expressions/DecimalUtil.java |    18 +
 .../vector/expressions/FilterExprAndExpr.java   |     8 +-
 .../vector/expressions/FilterExprOrExpr.java    |   140 +-
 ...FuncBRoundWithNumDigitsDecimalToDecimal.java |    40 +
 .../FuncRoundWithNumDigitsDecimalToDecimal.java |    14 +-
 .../ql/exec/vector/expressions/MathExpr.java    |    22 +
 .../hive/ql/exec/vector/expressions/NotCol.java |    14 +-
 .../ql/exec/vector/expressions/StringExpr.java  |    51 +
 ...VectorMapJoinInnerBigOnlyStringOperator.java |     4 +-
 .../VectorMapJoinInnerStringOperator.java       |     4 +-
 .../VectorMapJoinLeftSemiStringOperator.java    |     4 +-
 .../VectorMapJoinOuterStringOperator.java       |     4 +-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |     9 +-
 .../hadoop/hive/ql/hooks/LineageInfo.java       |     9 +-
 .../hadoop/hive/ql/hooks/LineageLogger.java     |    46 +-
 .../hive/ql/hooks/PostExecOrcFileDump.java      |   120 +
 .../ql/hooks/PostExecTezSummaryPrinter.java     |    72 +
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |   183 +-
 .../hive/ql/io/CombineHiveInputFormat.java      |    39 -
 .../apache/hadoop/hive/ql/io/orc/FileDump.java  |    52 +-
 .../apache/hadoop/hive/ql/io/orc/OrcConf.java   |     2 +-
 .../apache/hadoop/hive/ql/io/orc/OrcFile.java   |     3 +
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   447 +-
 .../apache/hadoop/hive/ql/io/orc/OrcSerde.java  |     7 +-
 .../apache/hadoop/hive/ql/io/orc/OrcSplit.java  |    25 +-
 .../apache/hadoop/hive/ql/io/orc/OrcStruct.java |     2 +-
 .../apache/hadoop/hive/ql/io/orc/Reader.java    |    13 +
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |   260 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |    24 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |    18 +-
 .../hive/ql/io/parquet/LeafFilterFactory.java   |    43 +-
 .../hive/ql/io/parquet/ProjectionPusher.java    |     3 +-
 .../parquet/read/DataWritableReadSupport.java   |    10 +-
 .../read/ParquetFilterPredicateConverter.java   |    35 +-
 .../serde/ParquetHiveArrayInspector.java        |    12 +
 .../ql/io/parquet/timestamp/NanoTimeUtils.java  |    23 +-
 .../ql/io/rcfile/stats/PartialScanTask.java     |    20 +-
 .../hive/ql/io/sarg/ConvertAstToSearchArg.java  |    21 +-
 .../hadoop/hive/ql/lib/DefaultGraphWalker.java  |    84 +-
 .../hadoop/hive/ql/lib/ForwardWalker.java       |    35 +-
 .../hadoop/hive/ql/lib/LevelOrderWalker.java    |   153 +
 .../hadoop/hive/ql/lib/PreOrderOnceWalker.java  |    44 +
 .../hadoop/hive/ql/lib/PreOrderWalker.java      |     2 +-
 .../apache/hadoop/hive/ql/lib/RuleRegExp.java   |    22 +-
 .../hadoop/hive/ql/lockmgr/DbLockManager.java   |    12 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |    37 +-
 .../hadoop/hive/ql/log/HiveEventCounter.java    |   135 +
 .../apache/hadoop/hive/ql/log/NullAppender.java |    63 +
 .../ql/log/PidDailyRollingFileAppender.java     |    33 -
 .../hive/ql/log/PidFilePatternConverter.java    |    62 +
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   178 +-
 .../hadoop/hive/ql/metadata/Partition.java      |     2 +-
 .../ql/metadata/SessionHiveMetaStoreClient.java |     2 +-
 .../hadoop/hive/ql/metadata/TableIterable.java  |   104 +
 .../hadoop/hive/ql/optimizer/ColumnPruner.java  |     8 +-
 .../ql/optimizer/ColumnPrunerProcFactory.java   |     3 +
 .../hive/ql/optimizer/ConstantPropagate.java    |    12 +-
 .../optimizer/ConstantPropagateProcFactory.java |   217 +-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |    22 +-
 .../hive/ql/optimizer/GenMapRedUtils.java       |    59 +-
 .../hive/ql/optimizer/GroupByOptimizer.java     |    58 +-
 .../ql/optimizer/IdentityProjectRemover.java    |    15 +
 .../hadoop/hive/ql/optimizer/IndexUtils.java    |    13 +-
 .../hadoop/hive/ql/optimizer/Optimizer.java     |    13 +
 .../hive/ql/optimizer/PointLookupOptimizer.java |   378 +
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |    19 +-
 .../ql/optimizer/calcite/HiveRelOptUtil.java    |    23 -
 .../calcite/reloperators/HiveFilter.java        |     2 +-
 .../calcite/reloperators/HiveLimit.java         |    57 -
 .../calcite/reloperators/HiveSort.java          |    85 -
 .../calcite/reloperators/HiveSortLimit.java     |   110 +
 .../rules/HiveAggregateProjectMergeRule.java    |   151 +
 .../rules/HiveJoinProjectTransposeRule.java     |   238 +-
 .../calcite/rules/HiveRelFieldTrimmer.java      |   145 +-
 .../calcite/stats/HiveRelMdMemory.java          |     9 +-
 .../calcite/stats/HiveRelMdParallelism.java     |     4 +-
 .../calcite/translator/ASTConverter.java        |    24 +-
 .../calcite/translator/ExprNodeConverter.java   |    70 +-
 .../calcite/translator/HiveOpConverter.java     |   110 +-
 .../translator/PlanModifierForASTConv.java      |    14 +-
 .../translator/PlanModifierForReturnPath.java   |     4 -
 .../calcite/translator/PlanModifierUtil.java    |     4 +-
 .../calcite/translator/TypeConverter.java       |    13 +-
 .../correlation/AbstractCorrelationProcCtx.java |     7 +
 .../correlation/CorrelationUtilities.java       |    11 +-
 .../correlation/ReduceSinkDeDuplication.java    |     6 +-
 .../ql/optimizer/index/RewriteCanApplyCtx.java  |     8 +-
 .../ql/optimizer/lineage/ExprProcFactory.java   |     9 +-
 .../hive/ql/optimizer/lineage/Generator.java    |     4 +-
 .../hive/ql/optimizer/lineage/LineageCtx.java   |    34 +-
 .../ql/optimizer/lineage/OpProcFactory.java     |    10 +-
 .../annotation/AnnotateWithOpTraits.java        |     6 +-
 .../ql/optimizer/pcr/PcrExprProcFactory.java    |   154 +-
 .../physical/NullScanTaskDispatcher.java        |     6 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |    15 +-
 .../hive/ql/optimizer/ppr/OpProcFactory.java    |     3 +-
 .../hive/ql/optimizer/ppr/PartitionPruner.java  |    69 +-
 .../annotation/AnnotateWithStatistics.java      |     6 +-
 .../stats/annotation/StatsRulesProcFactory.java |    46 +-
 .../ql/optimizer/unionproc/UnionProcessor.java  |    10 +-
 .../apache/hadoop/hive/ql/parse/ASTNode.java    |   139 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    73 +-
 .../ql/parse/ColumnStatsSemanticAnalyzer.java   |     6 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |    11 +
 .../hadoop/hive/ql/parse/GenMapRedWalker.java   |     2 +-
 .../hadoop/hive/ql/parse/GenTezWorkWalker.java  |     2 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |     2 +
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |    62 +-
 .../hadoop/hive/ql/parse/LeadLagInfo.java       |     4 +-
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |    38 +-
 .../apache/hadoop/hive/ql/parse/ParseUtils.java |     2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    54 +-
 .../apache/hadoop/hive/ql/parse/TezWalker.java  |     2 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |    42 +-
 .../hive/ql/parse/spark/GenSparkWorkWalker.java |     2 +-
 .../hive/ql/plan/ExprNodeConstantDesc.java      |    29 +-
 .../hadoop/hive/ql/plan/ExprNodeDesc.java       |    23 +-
 .../apache/hadoop/hive/ql/plan/FilterDesc.java  |    14 +-
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |    10 -
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |     9 +-
 .../hadoop/hive/ql/ppd/ExprWalkerInfo.java      |   136 +-
 .../hive/ql/ppd/ExprWalkerProcFactory.java      |    92 +-
 .../hadoop/hive/ql/ppd/OpProcFactory.java       |    11 +-
 .../ql/ppd/PredicateTransitivePropagate.java    |     4 +-
 .../hive/ql/ppd/SyntheticJoinPredicate.java     |     4 +-
 .../hadoop/hive/ql/processors/SetProcessor.java |     4 +
 .../sqlstd/SQLStdHiveAccessController.java      |     5 +
 .../hadoop/hive/ql/session/SessionState.java    |    10 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |   126 +-
 .../hive/ql/txn/AcidHouseKeeperService.java     |   104 +
 .../hive/ql/txn/compactor/CompactorMR.java      |    19 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |    10 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java    |     2 +-
 .../hive/ql/udf/generic/GenericUDAFMax.java     |    16 +-
 .../hive/ql/udf/generic/GenericUDAFStd.java     |     2 +
 .../ql/udf/generic/GenericUDAFVariance.java     |     2 +
 .../hadoop/hive/ql/udf/generic/GenericUDF.java  |    14 +-
 .../hive/ql/udf/generic/GenericUDFAesBase.java  |   205 +
 .../ql/udf/generic/GenericUDFAesDecrypt.java    |    50 +
 .../ql/udf/generic/GenericUDFAesEncrypt.java    |    50 +
 .../hive/ql/udf/generic/GenericUDFBRound.java   |    68 +
 .../ql/udf/generic/GenericUDFBaseNumeric.java   |     4 +-
 .../hive/ql/udf/generic/GenericUDFBasePad.java  |     8 +-
 .../hive/ql/udf/generic/GenericUDFIn.java       |    14 +-
 .../hive/ql/udf/generic/GenericUDFNvl.java      |     2 +-
 .../hive/ql/udf/generic/GenericUDFOPAnd.java    |    63 +-
 .../hive/ql/udf/generic/GenericUDFOPEqual.java  |     4 +
 .../generic/GenericUDFOPEqualOrGreaterThan.java |     4 +
 .../generic/GenericUDFOPEqualOrLessThan.java    |     4 +
 .../ql/udf/generic/GenericUDFOPGreaterThan.java |     4 +
 .../ql/udf/generic/GenericUDFOPLessThan.java    |     4 +
 .../ql/udf/generic/GenericUDFOPNotEqual.java    |     5 +
 .../ql/udf/generic/GenericUDFOPNotNull.java     |     4 +
 .../hive/ql/udf/generic/GenericUDFOPNull.java   |     4 +
 .../hive/ql/udf/generic/GenericUDFOPOr.java     |    63 +-
 .../ql/udf/generic/GenericUDFParamUtils.java    |     8 +-
 .../hive/ql/udf/generic/GenericUDFRound.java    |    41 +-
 .../hive/ql/udf/generic/GenericUDFStruct.java   |    25 +-
 .../hadoop/hive/ql/udf/generic/RoundUtils.java  |    14 +
 .../main/resources/hive-exec-log4j.properties   |    77 -
 ql/src/main/resources/hive-exec-log4j2.xml      |   109 +
 ql/src/main/resources/tez-container-log4j2.xml  |    48 +
 .../apache/hadoop/hive/ql/TestTxnCommands.java  |    21 +
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |    84 +-
 .../hadoop/hive/ql/exec/TestOperators.java      |    16 +
 .../exec/vector/TestVectorizationContext.java   |    93 +
 .../TestVectorLogicalExpressions.java           |   282 +
 .../apache/hadoop/hive/ql/io/TestAcidUtils.java |    27 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |   118 +-
 .../hadoop/hive/ql/io/orc/TestOrcFile.java      |    10 +-
 .../hadoop/hive/ql/io/orc/TestOrcStruct.java    |     2 +
 .../hive/ql/io/orc/TestRecordReaderImpl.java    |    42 +-
 .../parquet/TestParquetRecordReaderWrapper.java |    50 +-
 .../read/TestParquetFilterPredicate.java        |    27 +-
 .../serde/TestParquetTimestampUtils.java        |    38 +-
 .../ql/io/sarg/TestConvertAstToSearchArg.java   |   128 +-
 .../hive/ql/io/sarg/TestSearchArgumentImpl.java |    22 +-
 .../hive/ql/lockmgr/TestDbTxnManager.java       |    74 +-
 .../hive/ql/lockmgr/TestDbTxnManager2.java      |     2 +-
 .../hadoop/hive/ql/log/TestLog4j2Appenders.java |    95 +
 .../hadoop/hive/ql/metadata/StringAppender.java |   128 +
 .../hadoop/hive/ql/metadata/TestHive.java       |    50 +-
 ...nMapRedUtilsUsePartitionColumnsNegative.java |    73 +
 ...nMapRedUtilsUsePartitionColumnsPositive.java |    61 +
 .../TestSQL11ReservedKeyWordsNegative.java      |    32 +-
 .../TestSQL11ReservedKeyWordsPositive.java      |    23 +-
 .../udf/generic/TestGenericUDFAesDecrypt.java   |   233 +
 .../udf/generic/TestGenericUDFAesEncrypt.java   |   228 +
 .../ql/udf/generic/TestGenericUDFBRound.java    |   202 +
 .../clientnegative/alter_table_wrong_location.q |     4 +
 .../queries/clientnegative/ctas_noemptyfolder.q |    10 +
 .../queries/clientnegative/load_orc_negative3.q |     6 +
 .../clientnegative/mismatch_columns_insertion.q |     4 +
 .../queries/clientnegative/nvl_mismatch_type.q  |    20 +
 .../annotate_stats_deep_filters.q               |    67 +
 .../clientpositive/authorization_1_sql_std.q    |     4 +
 .../authorization_set_show_current_role.q       |     3 +
 .../clientpositive/bucket_map_join_tez1.q       |    31 +
 .../clientpositive/cast_tinyint_to_double.q     |     7 +
 .../queries/clientpositive/cbo_rp_auto_join17.q |    14 +
 .../cbo_rp_cross_product_check_2.q              |    31 +
 ql/src/test/queries/clientpositive/cbo_rp_gby.q |    24 +
 .../queries/clientpositive/cbo_rp_gby_empty.q   |    30 +
 .../test/queries/clientpositive/cbo_rp_insert.q |    17 +
 .../test/queries/clientpositive/cbo_rp_join.q   |    65 +
 .../test/queries/clientpositive/cbo_rp_limit.q  |    16 +
 .../clientpositive/cbo_rp_outer_join_ppr.q      |    40 +
 .../queries/clientpositive/cbo_rp_semijoin.q    |    17 +
 .../clientpositive/cbo_rp_simple_select.q       |    56 +
 .../test/queries/clientpositive/cbo_rp_stats.q  |    10 +
 .../queries/clientpositive/cbo_rp_subq_exists.q |    67 +
 .../queries/clientpositive/cbo_rp_subq_in.q     |    56 +
 .../queries/clientpositive/cbo_rp_subq_not_in.q |    81 +
 .../queries/clientpositive/cbo_rp_udf_udaf.q    |    20 +
 .../test/queries/clientpositive/cbo_rp_union.q  |    14 +
 .../test/queries/clientpositive/cbo_rp_views.q  |    46 +
 .../queries/clientpositive/cbo_rp_windowing.q   |    21 +
 .../queries/clientpositive/cbo_rp_windowing_2.q |   439 +
 ql/src/test/queries/clientpositive/char_udf1.q  |     9 +-
 .../clientpositive/columnstats_quoting.q        |     8 +
 .../queries/clientpositive/compustat_avro.q     |     8 +-
 .../test/queries/clientpositive/create_like.q   |    12 +
 .../test/queries/clientpositive/dynpart_merge.q |    28 +
 .../queries/clientpositive/exchgpartition2lel.q |    32 +
 .../queries/clientpositive/flatten_and_or.q     |    19 +
 .../queries/clientpositive/folder_predicate.q   |    32 +
 ql/src/test/queries/clientpositive/groupby13.q  |    16 +
 .../queries/clientpositive/groupby1_map_nomap.q |     2 +
 ql/src/test/queries/clientpositive/groupby6.q   |     2 +
 .../clientpositive/groupby_grouping_id2.q       |     2 +
 .../clientpositive/groupby_ppr_multi_distinct.q |     2 +
 ql/src/test/queries/clientpositive/having2.q    |    27 +
 .../clientpositive/insertoverwrite_bucket.q     |    28 +
 ql/src/test/queries/clientpositive/keyword_2.q  |    14 +
 ql/src/test/queries/clientpositive/lineage3.q   |    22 +-
 .../clientpositive/load_dyn_part14_win.q        |    18 +-
 .../test/queries/clientpositive/load_orc_part.q |     4 +
 .../queries/clientpositive/macro_duplicate.q    |    10 +
 .../queries/clientpositive/multi_column_in.q    |    71 +
 .../test/queries/clientpositive/orc_file_dump.q |    57 +
 .../test/queries/clientpositive/orc_ppd_basic.q |   177 +
 .../parquet_mixed_partition_formats.q           |    42 +
 .../clientpositive/parquet_ppd_boolean.q        |    35 +
 .../queries/clientpositive/parquet_ppd_char.q   |    76 +
 .../queries/clientpositive/parquet_ppd_date.q   |   101 +
 .../clientpositive/parquet_ppd_decimal.q        |   163 +
 .../clientpositive/parquet_ppd_partition.q      |     9 +
 .../clientpositive/parquet_ppd_timestamp.q      |    98 +
 .../clientpositive/parquet_ppd_varchar.q        |    76 +
 .../clientpositive/parquet_predicate_pushdown.q |   301 +-
 .../test/queries/clientpositive/pointlookup.q   |    59 +
 .../test/queries/clientpositive/pointlookup2.q  |    51 +
 .../test/queries/clientpositive/pointlookup3.q  |    41 +
 .../queries/clientpositive/ptfgroupbyjoin.q     |    61 +
 .../queries/clientpositive/selectDistinctStar.q |     2 +
 ql/src/test/queries/clientpositive/structin.q   |    23 +
 .../queries/clientpositive/udf_aes_decrypt.q    |    21 +
 .../queries/clientpositive/udf_aes_encrypt.q    |    21 +
 ql/src/test/queries/clientpositive/udf_bround.q |    44 +
 .../clientpositive/unionall_unbalancedppd.q     |     3 +
 .../test/queries/clientpositive/varchar_udf1.q  |     6 +-
 .../test/queries/clientpositive/vector_bround.q |    14 +
 .../clientpositive/vector_cast_constant.q       |     4 +-
 .../queries/clientpositive/vectorized_casts.q   |     6 +
 .../queries/clientpositive/windowing_udaf.q     |     4 +
 .../alter_table_wrong_location.q.out            |     9 +
 .../clientnegative/char_pad_convert_fail0.q.out |     2 +-
 .../clientnegative/char_pad_convert_fail1.q.out |     2 +-
 .../clientnegative/char_pad_convert_fail3.q.out |     2 +-
 .../clientnegative/ctas_noemptyfolder.q.out     |    19 +
 .../clientnegative/load_orc_negative3.q.out     |    25 +
 .../mismatch_columns_insertion.q.out            |     9 +
 .../clientnegative/nvl_mismatch_type.q.out      |    43 +
 .../alter_partition_coltype.q.out               |     8 +-
 .../annotate_stats_deep_filters.q.out           |   244 +
 .../clientpositive/annotate_stats_filter.q.out  |    10 +-
 .../clientpositive/annotate_stats_groupby.q.out |   106 +-
 .../annotate_stats_groupby2.q.out               |    28 +-
 .../authorization_1_sql_std.q.out               |    11 +
 .../authorization_explain.q.java1.7.out         |     2 +-
 .../authorization_explain.q.java1.8.out         |     2 +-
 .../authorization_set_show_current_role.q.out   |     8 +
 .../results/clientpositive/auto_join18.q.out    |    12 +-
 .../auto_join18_multi_distinct.q.out            |    12 +-
 .../results/clientpositive/auto_join27.q.out    |    18 +-
 .../results/clientpositive/auto_join32.q.out    |     4 +-
 .../clientpositive/binarysortable_1.q.out       |   Bin 4329 -> 4325 bytes
 .../clientpositive/cast_tinyint_to_double.q.out |    38 +
 .../clientpositive/cbo_rp_auto_join17.q.out     |   118 +
 .../cbo_rp_cross_product_check_2.q.out          |   699 +
 .../results/clientpositive/cbo_rp_gby.q.out     |   124 +
 .../clientpositive/cbo_rp_gby_empty.q.out       |    77 +
 .../results/clientpositive/cbo_rp_insert.q.out  |    89 +
 .../results/clientpositive/cbo_rp_join.q.out    | 15028 +++++++++++++++++
 .../results/clientpositive/cbo_rp_limit.q.out   |    90 +
 .../cbo_rp_outer_join_ppr.q.java1.7.out         |   855 +
 .../clientpositive/cbo_rp_semijoin.q.out        |   440 +
 .../clientpositive/cbo_rp_simple_select.q.out   |   755 +
 .../results/clientpositive/cbo_rp_stats.q.out   |    14 +
 .../clientpositive/cbo_rp_subq_exists.q.out     |   297 +
 .../results/clientpositive/cbo_rp_subq_in.q.out |   151 +
 .../clientpositive/cbo_rp_subq_not_in.q.out     |   365 +
 .../clientpositive/cbo_rp_udf_udaf.q.out        |   125 +
 .../results/clientpositive/cbo_rp_union.q.out   |   920 +
 .../results/clientpositive/cbo_rp_views.q.out   |   237 +
 .../clientpositive/cbo_rp_windowing.q.out       |   293 +
 .../clientpositive/cbo_rp_windowing_2.q.out     |  2338 +++
 .../clientpositive/char_udf1.q.java1.7.out      |    22 +-
 .../clientpositive/columnstats_quoting.q.out    |   114 +
 .../results/clientpositive/compustat_avro.q.out |     8 +-
 .../clientpositive/constprog_partitioner.q.out  |    30 +-
 .../clientpositive/convert_enum_to_string.q.out |     9 +-
 .../clientpositive/correlationoptimizer10.q.out |    48 +-
 .../clientpositive/correlationoptimizer2.q.out  |   220 +-
 .../clientpositive/correlationoptimizer5.q.out  |     6 +-
 .../clientpositive/correlationoptimizer6.q.out  |   232 +-
 ql/src/test/results/clientpositive/count.q.out  |    14 +-
 .../results/clientpositive/create_like.q.out    |    66 +
 .../results/clientpositive/ctas_colname.q.out   |    52 +-
 .../test/results/clientpositive/database.q.out  |     2 +-
 .../clientpositive/decimal_precision.q.out      |     4 +-
 .../results/clientpositive/decimal_udf.q.out    |    48 +-
 .../results/clientpositive/distinct_stats.q.out |    14 +-
 .../clientpositive/dynamic_rdd_cache.q.out      |    58 +-
 .../results/clientpositive/dynpart_merge.q.out  |    99 +
 .../dynpart_sort_opt_vectorization.q.out        |   105 +-
 .../dynpart_sort_optimization.q.out             |   105 +-
 ...ryption_select_read_only_encrypted_tbl.q.out |     4 +-
 .../clientpositive/exchgpartition2lel.q.out     |   182 +
 .../clientpositive/explain_dependency.q.out     |    18 +-
 .../clientpositive/explain_dependency2.q.out    |    16 +-
 .../clientpositive/explain_logical.q.out        |    78 +-
 .../clientpositive/fetch_aggregation.q.out      |     4 +-
 .../clientpositive/filter_cond_pushdown.q.out   |    32 +-
 .../clientpositive/filter_join_breaktask.q.out  |    12 +-
 .../results/clientpositive/flatten_and_or.q.out |    66 +
 .../test/results/clientpositive/fold_when.q.out |    16 +-
 .../clientpositive/folder_predicate.q.out       |   368 +
 .../test/results/clientpositive/gby_star.q.out  |    54 +-
 .../test/results/clientpositive/groupby12.q.out |     6 +-
 .../test/results/clientpositive/groupby13.q.out |    86 +
 .../clientpositive/groupby1_map_nomap.q.out     |     8 +-
 .../results/clientpositive/groupby5_map.q.out   |     4 +-
 .../clientpositive/groupby5_map_skew.q.out      |     4 +-
 .../test/results/clientpositive/groupby6.q.out  |     8 +-
 .../results/clientpositive/groupby_cube1.q.out  |    12 +-
 .../groupby_distinct_samekey.q.out              |     6 +-
 .../clientpositive/groupby_duplicate_key.q.out  |    16 +-
 .../clientpositive/groupby_grouping_id2.q.out   |    28 +-
 .../clientpositive/groupby_grouping_sets2.q.out |    10 +-
 .../clientpositive/groupby_grouping_sets3.q.out |    12 +-
 .../clientpositive/groupby_grouping_sets5.q.out |     8 +-
 .../clientpositive/groupby_grouping_sets6.q.out |     8 +-
 .../groupby_multi_single_reducer2.q.out         |     2 +-
 .../groupby_multi_single_reducer3.q.out         |    12 +-
 .../clientpositive/groupby_position.q.out       |    36 +-
 .../groupby_ppr_multi_distinct.q.out            |     8 +-
 .../clientpositive/groupby_resolution.q.out     |    60 +-
 .../clientpositive/groupby_rollup1.q.out        |    12 +-
 .../clientpositive/groupby_sort_10.q.out        |     8 +-
 .../clientpositive/groupby_sort_11.q.out        |    10 +-
 .../clientpositive/groupby_sort_1_23.q.out      |    56 +-
 .../results/clientpositive/groupby_sort_8.q.out |    12 +-
 .../clientpositive/groupby_sort_skew_1_23.q.out |    56 +-
 ql/src/test/results/clientpositive/having.q.out |    62 +-
 .../test/results/clientpositive/having2.q.out   |   365 +-
 .../clientpositive/index_auto_mult_tables.q.out |    12 +-
 .../clientpositive/index_auto_self_join.q.out   |    12 +-
 .../clientpositive/index_auto_update.q.out      |     6 +-
 .../index_bitmap_auto_partitioned.q.out         |     6 +-
 .../index_bitmap_compression.q.out              |     6 +-
 .../infer_bucket_sort_dyn_part.q.out            |     4 +-
 .../infer_bucket_sort_map_operators.q.out       |     4 +-
 .../clientpositive/infer_const_type.q.out       |     7 +-
 .../results/clientpositive/input4.q.java1.7.out |     2 +-
 .../results/clientpositive/input4.q.java1.8.out |     2 +-
 .../clientpositive/input_testxpath2.q.out       |     2 +-
 .../clientpositive/input_testxpath4.q.out       |     2 +-
 .../clientpositive/insertoverwrite_bucket.q.out |   104 +
 .../results/clientpositive/join0.q.java1.7.out  |     2 +-
 .../results/clientpositive/join0.q.java1.8.out  |     4 +-
 ql/src/test/results/clientpositive/join18.q.out |    12 +-
 .../clientpositive/join18_multi_distinct.q.out  |    12 +-
 ql/src/test/results/clientpositive/join31.q.out |    36 +-
 ql/src/test/results/clientpositive/join32.q.out |     2 +-
 .../clientpositive/join32_lessSize.q.out        |     6 +-
 ql/src/test/results/clientpositive/join33.q.out |     2 +-
 .../join_cond_pushdown_unqual4.q.out            |     2 +-
 .../test/results/clientpositive/keyword_2.q.out |    51 +
 .../limit_partition_metadataonly.q.out          |     4 +-
 .../results/clientpositive/limit_pushdown.q.out |    36 +-
 .../test/results/clientpositive/lineage2.q.out  |    12 +-
 .../test/results/clientpositive/lineage3.q.out  |    79 +-
 .../list_bucket_dml_6.q.java1.7.out             |    12 +-
 .../list_bucket_dml_6.q.java1.8.out             |    12 +-
 .../clientpositive/list_bucket_dml_7.q.out      |    12 +-
 .../list_bucket_query_multiskew_3.q.out         |     2 +-
 .../list_bucket_query_oneskew_3.q.out           |     6 +-
 .../clientpositive/load_dyn_part14_win.q.out    |   167 +-
 .../results/clientpositive/load_orc_part.q.out  |    18 +
 .../clientpositive/macro_duplicate.q.out        |    56 +
 .../clientpositive/mapjoin_mapjoin.q.out        |    32 +-
 .../clientpositive/metadata_only_queries.q.out  |     4 +-
 .../results/clientpositive/metadataonly1.q.out  |   112 +-
 .../results/clientpositive/multiMapJoin2.q.out  |   226 +-
 .../clientpositive/multi_column_in.q.out        |   410 +
 .../results/clientpositive/multi_insert.q.out   |     8 +-
 .../clientpositive/multi_insert_gby.q.out       |     2 +-
 .../multi_insert_lateral_view.q.out             |     4 +-
 ...i_insert_move_tasks_share_dependencies.q.out |   360 +-
 .../nonblock_op_deduplicate.q.out               |     8 +-
 .../results/clientpositive/nonmr_fetch.q.out    |    14 +-
 .../test/results/clientpositive/null_cast.q.out |     6 +-
 .../results/clientpositive/orc_file_dump.q.out  |   447 +
 .../clientpositive/orc_predicate_pushdown.q.out |    36 +-
 .../results/clientpositive/parallel_join0.q.out |     2 +-
 .../parquet_mixed_partition_formats.q.out       |   303 +
 .../clientpositive/parquet_ppd_boolean.q.out    |   270 +
 .../clientpositive/parquet_ppd_char.q.out       |   308 +
 .../clientpositive/parquet_ppd_date.q.out       |   435 +
 .../clientpositive/parquet_ppd_decimal.q.out    |   768 +
 .../clientpositive/parquet_ppd_partition.q.out  |    47 +
 .../clientpositive/parquet_ppd_timestamp.q.out  |   422 +
 .../clientpositive/parquet_ppd_varchar.q.out    |   308 +
 .../parquet_predicate_pushdown.q.out            |  1307 +-
 .../clientpositive/partition_multilevels.q.out  |     8 +-
 .../clientpositive/plan_json.q.java1.7.out      |     2 +-
 .../clientpositive/plan_json.q.java1.8.out      |     2 +-
 .../results/clientpositive/pointlookup.q.out    |   198 +
 .../results/clientpositive/pointlookup2.q.out   |  1647 ++
 .../results/clientpositive/pointlookup3.q.out   |  1394 ++
 .../test/results/clientpositive/ppd_gby.q.out   |    12 +-
 .../test/results/clientpositive/ppd_gby2.q.out  |    60 +-
 .../results/clientpositive/ppd_gby_join.q.out   |     4 +-
 .../test/results/clientpositive/ppd_join.q.out  |     4 +-
 .../test/results/clientpositive/ppd_join2.q.out |    22 +-
 .../test/results/clientpositive/ppd_join3.q.out |    52 +-
 .../clientpositive/ppd_join_filter.q.out        |    98 +-
 .../clientpositive/ppd_outer_join4.q.out        |     2 +-
 ql/src/test/results/clientpositive/ptf.q.out    |    27 +-
 .../results/clientpositive/ptfgroupbyjoin.q.out |   519 +
 .../ql_rewrite_gbtoidx_cbo_1.q.out              |   168 +-
 .../ql_rewrite_gbtoidx_cbo_2.q.out              |    94 +-
 .../clientpositive/rand_partitionpruner3.q.out  |    12 +-
 .../reduce_deduplicate_extended.q.out           |    32 +-
 .../clientpositive/selectDistinctStar.q.out     |    52 +-
 .../clientpositive/select_unquote_not.q.out     |     8 +-
 .../results/clientpositive/show_functions.q.out |     3 +
 .../clientpositive/spark/auto_join18.q.out      |    10 +-
 .../spark/auto_join18_multi_distinct.q.out      |    12 +-
 .../clientpositive/spark/auto_join27.q.out      |    18 +-
 .../clientpositive/spark/auto_join32.q.out      |    53 +-
 .../spark/bucket_map_join_tez1.q.out            |   357 +
 .../spark/constprog_partitioner.q.out           |    30 +-
 .../results/clientpositive/spark/count.q.out    |    14 +-
 .../spark/dynamic_rdd_cache.q.out               |     6 +-
 .../spark/filter_join_breaktask.q.out           |    12 +-
 .../spark/groupby1_map_nomap.q.out              |   564 +-
 .../clientpositive/spark/groupby5_map.q.out     |     4 +-
 .../spark/groupby5_map_skew.q.out               |     4 +-
 .../results/clientpositive/spark/groupby6.q.out |    20 +-
 .../clientpositive/spark/groupby_cube1.q.out    |    12 +-
 .../spark/groupby_grouping_id2.q.out            |    38 +-
 .../spark/groupby_multi_single_reducer2.q.out   |     2 +-
 .../spark/groupby_multi_single_reducer3.q.out   |    12 +-
 .../clientpositive/spark/groupby_position.q.out |    18 +-
 .../spark/groupby_ppr_multi_distinct.q.out      |    16 +-
 .../spark/groupby_resolution.q.out              |    60 +-
 .../clientpositive/spark/groupby_rollup1.q.out  |    12 +-
 .../spark/groupby_sort_1_23.q.out               |    90 +-
 .../spark/groupby_sort_skew_1_23.q.out          |    90 +-
 .../results/clientpositive/spark/having.q.out   |    62 +-
 .../spark/infer_bucket_sort_map_operators.q.out |     4 +-
 .../results/clientpositive/spark/join18.q.out   |    10 +-
 .../spark/join18_multi_distinct.q.out           |    12 +-
 .../results/clientpositive/spark/join31.q.out   |    36 +-
 .../results/clientpositive/spark/join32.q.out   |     2 +-
 .../clientpositive/spark/join32_lessSize.q.out  |     6 +-
 .../results/clientpositive/spark/join33.q.out   |     2 +-
 .../spark/join_cond_pushdown_unqual4.q.out      |     2 +-
 .../spark/limit_partition_metadataonly.q.out    |     4 +-
 .../clientpositive/spark/limit_pushdown.q.out   |    34 +-
 .../clientpositive/spark/mapjoin_mapjoin.q.out  |    24 +-
 .../spark/metadata_only_queries.q.out           |     4 +-
 .../clientpositive/spark/multi_insert.q.out     |     8 +-
 .../clientpositive/spark/multi_insert_gby.q.out |     2 +-
 .../spark/multi_insert_lateral_view.q.out       |     4 +-
 ...i_insert_move_tasks_share_dependencies.q.out |   536 +-
 .../clientpositive/spark/ppd_gby_join.q.out     |     4 +-
 .../results/clientpositive/spark/ppd_join.q.out |     4 +-
 .../clientpositive/spark/ppd_join2.q.out        |    22 +-
 .../clientpositive/spark/ppd_join3.q.out        |    52 +-
 .../clientpositive/spark/ppd_join_filter.q.out  |    90 +-
 .../clientpositive/spark/ppd_outer_join4.q.out  |     2 +-
 .../test/results/clientpositive/spark/ptf.q.out |    17 +-
 .../spark/ql_rewrite_gbtoidx_cbo_1.q.out        |   168 +-
 .../clientpositive/spark/stats_only_null.q.out  |     8 +-
 .../clientpositive/spark/subquery_exists.q.out  |    12 +-
 .../clientpositive/spark/subquery_in.q.out      |    72 +-
 .../spark/subquery_multiinsert.q.java1.7.out    |    16 +-
 .../results/clientpositive/spark/union11.q.out  |    42 +-
 .../results/clientpositive/spark/union14.q.out  |    28 +-
 .../results/clientpositive/spark/union15.q.out  |    28 +-
 .../results/clientpositive/spark/union28.q.out  |     4 +-
 .../results/clientpositive/spark/union30.q.out  |     4 +-
 .../results/clientpositive/spark/union33.q.out  |     8 +-
 .../results/clientpositive/spark/union5.q.out   |    34 +-
 .../results/clientpositive/spark/union7.q.out   |    28 +-
 .../clientpositive/spark/union_remove_21.q.out  |     4 +-
 .../spark/union_remove_6_subq.q.out             |    22 +-
 .../spark/vector_cast_constant.q.java1.7.out    |    41 +-
 .../spark/vector_count_distinct.q.out           |     4 +-
 .../spark/vector_decimal_aggregate.q.out        |    12 +-
 .../spark/vector_distinct_2.q.out               |    28 +-
 .../clientpositive/spark/vector_groupby_3.q.out |    30 +-
 .../spark/vector_mapjoin_reduce.q.out           |    58 +-
 .../clientpositive/spark/vector_orderby_5.q.out |     6 +-
 .../clientpositive/spark/vectorization_0.q.out  |    18 +-
 .../clientpositive/spark/vectorization_13.q.out |    36 +-
 .../clientpositive/spark/vectorization_15.q.out |    18 +-
 .../clientpositive/spark/vectorization_16.q.out |    16 +-
 .../clientpositive/spark/vectorization_17.q.out |    12 +-
 .../clientpositive/spark/vectorization_9.q.out  |    16 +-
 .../spark/vectorization_pushdown.q.out          |     4 +-
 .../spark/vectorization_short_regress.q.out     |   114 +-
 .../clientpositive/spark/vectorized_case.q.out  |     2 +-
 .../spark/vectorized_nested_mapjoin.q.out       |    18 +-
 .../clientpositive/spark/vectorized_ptf.q.out   |    21 +-
 .../spark/vectorized_timestamp_funcs.q.out      |    12 +-
 .../clientpositive/stats_only_null.q.out        |     8 +-
 .../results/clientpositive/stats_ppr_all.q.out  |    16 +-
 .../test/results/clientpositive/structin.q.out  |   110 +
 .../subq_where_serialization.q.out              |    18 +-
 .../clientpositive/subquery_exists.q.out        |    12 +-
 .../clientpositive/subquery_exists_having.q.out |    48 +-
 .../results/clientpositive/subquery_in.q.out    |    72 +-
 .../clientpositive/subquery_in_having.q.out     |   310 +-
 .../subquery_multiinsert.q.java1.7.out          |    16 +-
 .../clientpositive/subquery_notexists.q.out     |    18 +-
 .../subquery_notexists_having.q.out             |    26 +-
 .../results/clientpositive/subquery_notin.q.out |    24 +-
 .../subquery_notin_having.q.java1.7.out         |    50 +-
 .../subquery_unqualcolumnrefs.q.out             |   100 +-
 .../results/clientpositive/subquery_views.q.out |    40 +-
 .../tez/bucket_map_join_tez1.q.out              |   333 +
 .../clientpositive/tez/constprog_dpp.q.out      |     4 +-
 .../test/results/clientpositive/tez/count.q.out |    14 +-
 .../tez/dynamic_partition_pruning.q.out         |    88 +-
 .../tez/dynpart_sort_opt_vectorization.q.out    |    90 +-
 .../tez/dynpart_sort_optimization.q.out         |    89 +-
 .../clientpositive/tez/explainuser_1.q.out      |  3083 ++--
 .../clientpositive/tez/explainuser_2.q.out      |  4016 ++---
 .../clientpositive/tez/explainuser_3.q.out      |    10 +-
 .../tez/filter_join_breaktask.q.out             |    12 +-
 .../results/clientpositive/tez/having.q.out     |    62 +-
 .../clientpositive/tez/limit_pushdown.q.out     |    34 +-
 .../clientpositive/tez/mapjoin_mapjoin.q.out    |    24 +-
 .../tez/metadata_only_queries.q.out             |     4 +-
 .../clientpositive/tez/metadataonly1.q.out      |    44 +-
 .../test/results/clientpositive/tez/mrr.q.out   |    94 +-
 .../clientpositive/tez/orc_ppd_basic.q.out      |   701 +
 .../test/results/clientpositive/tez/ptf.q.out   |    15 +-
 .../clientpositive/tez/selectDistinctStar.q.out |    52 +-
 .../clientpositive/tez/stats_only_null.q.out    |     8 +-
 .../clientpositive/tez/subquery_exists.q.out    |    12 +-
 .../clientpositive/tez/subquery_in.q.out        |    72 +-
 .../results/clientpositive/tez/tez_dml.q.out    |     6 +-
 .../results/clientpositive/tez/union5.q.out     |    44 +-
 .../results/clientpositive/tez/union7.q.out     |    28 +-
 .../clientpositive/tez/unionDistinct_1.q.out    |     8 +-
 .../clientpositive/tez/vector_aggregate_9.q.out |     4 +-
 .../tez/vector_binary_join_groupby.q.out        |     4 +-
 .../tez/vector_cast_constant.q.java1.7.out      |    21 +-
 .../tez/vector_char_mapjoin1.q.out              |     1 +
 .../tez/vector_count_distinct.q.out             |     4 +-
 .../tez/vector_decimal_aggregate.q.out          |    12 +-
 .../tez/vector_decimal_precision.q.out          |     4 +-
 .../clientpositive/tez/vector_decimal_udf.q.out |    54 +-
 .../clientpositive/tez/vector_distinct_2.q.out  |    28 +-
 .../clientpositive/tez/vector_groupby_3.q.out   |    30 +-
 .../tez/vector_groupby_reduce.q.out             |     8 +-
 .../tez/vector_grouping_sets.q.out              |     8 +-
 .../clientpositive/tez/vector_inner_join.q.out  |    14 +-
 .../tez/vector_mapjoin_reduce.q.out             |    60 +-
 .../tez/vector_mr_diff_schema_alias.q.out       |     2 +-
 .../clientpositive/tez/vector_orderby_5.q.out   |     6 +-
 .../clientpositive/tez/vector_outer_join2.q.out |    20 +-
 .../tez/vector_partition_diff_num_cols.q.out    |    20 +-
 .../tez/vector_partitioned_date_time.q.out      |    12 +-
 .../tez/vector_reduce_groupby_decimal.q.out     |    24 +-
 .../tez/vector_varchar_mapjoin1.q.out           |     1 +
 .../clientpositive/tez/vectorization_0.q.out    |    18 +-
 .../clientpositive/tez/vectorization_13.q.out   |    36 +-
 .../clientpositive/tez/vectorization_15.q.out   |    18 +-
 .../clientpositive/tez/vectorization_16.q.out   |    16 +-
 .../clientpositive/tez/vectorization_17.q.out   |    12 +-
 .../clientpositive/tez/vectorization_7.q.out    |     4 +-
 .../clientpositive/tez/vectorization_8.q.out    |     4 +-
 .../clientpositive/tez/vectorization_9.q.out    |    16 +-
 .../tez/vectorization_limit.q.out               |    14 +-
 .../tez/vectorization_pushdown.q.out            |     4 +-
 .../tez/vectorization_short_regress.q.out       |   114 +-
 .../clientpositive/tez/vectorized_case.q.out    |     2 +-
 .../clientpositive/tez/vectorized_casts.q.out   |    99 +-
 .../tez/vectorized_distinct_gby.q.out           |     8 +-
 .../vectorized_dynamic_partition_pruning.q.out  |    88 +-
 .../tez/vectorized_nested_mapjoin.q.out         |    18 +-
 .../clientpositive/tez/vectorized_parquet.q.out |     6 +-
 .../clientpositive/tez/vectorized_ptf.q.out     |    19 +-
 .../tez/vectorized_timestamp_funcs.q.out        |    12 +-
 ql/src/test/results/clientpositive/udf8.q.out   |     4 +-
 .../clientpositive/udf_aes_decrypt.q.out        |    79 +
 .../clientpositive/udf_aes_encrypt.q.out        |    79 +
 .../results/clientpositive/udf_bround.q.out     |   119 +
 .../test/results/clientpositive/udf_count.q.out |    16 +-
 .../results/clientpositive/udf_inline.q.out     |     8 +-
 .../clientpositive/udf_isnull_isnotnull.q.out   |     2 +-
 ql/src/test/results/clientpositive/udf_or.q.out |     4 +-
 .../test/results/clientpositive/udf_size.q.out  |     2 +-
 .../results/clientpositive/udf_struct.q.out     |     2 +-
 .../test/results/clientpositive/udf_union.q.out |     2 +-
 .../test/results/clientpositive/union11.q.out   |    70 +-
 .../test/results/clientpositive/union14.q.out   |    32 +-
 .../test/results/clientpositive/union15.q.out   |    38 +-
 .../test/results/clientpositive/union28.q.out   |     8 +-
 .../test/results/clientpositive/union30.q.out   |     8 +-
 .../test/results/clientpositive/union33.q.out   |     8 +-
 ql/src/test/results/clientpositive/union5.q.out |    48 +-
 ql/src/test/results/clientpositive/union7.q.out |    32 +-
 .../clientpositive/unionDistinct_1.q.out        |     8 +-
 .../clientpositive/union_remove_21.q.out        |     8 +-
 .../clientpositive/union_remove_6_subq.q.out    |    34 +-
 .../clientpositive/unionall_unbalancedppd.q.out |    14 +-
 .../clientpositive/varchar_udf1.q.java1.7.out   |    12 +-
 .../clientpositive/vector_aggregate_9.q.out     |     4 +-
 .../vector_aggregate_without_gby.q.out          |     4 +-
 .../vector_binary_join_groupby.q.out            |     4 +-
 .../results/clientpositive/vector_bround.q.out  |    86 +
 .../vector_cast_constant.q.java1.7.out          |    32 +-
 .../clientpositive/vector_char_mapjoin1.q.out   |     1 +
 .../clientpositive/vector_count_distinct.q.out  |     6 +-
 .../vector_decimal_aggregate.q.out              |    12 +-
 .../vector_decimal_precision.q.out              |     4 +-
 .../clientpositive/vector_decimal_udf.q.out     |    54 +-
 .../clientpositive/vector_distinct_2.q.out      |    28 +-
 .../clientpositive/vector_groupby_3.q.out       |    30 +-
 .../clientpositive/vector_groupby_reduce.q.out  |     8 +-
 .../clientpositive/vector_grouping_sets.q.out   |     8 +-
 .../clientpositive/vector_inner_join.q.out      |    12 +-
 .../clientpositive/vector_left_outer_join.q.out |     8 +-
 .../clientpositive/vector_mapjoin_reduce.q.out  |    62 +-
 .../vector_mr_diff_schema_alias.q.out           |     2 +-
 .../clientpositive/vector_orderby_5.q.out       |     6 +-
 .../clientpositive/vector_outer_join1.q.out     |     8 +-
 .../clientpositive/vector_outer_join2.q.out     |    28 +-
 .../clientpositive/vector_outer_join3.q.out     |    24 +-
 .../clientpositive/vector_outer_join4.q.out     |     8 +-
 .../clientpositive/vector_outer_join5.q.out     |    48 +-
 .../vector_partition_diff_num_cols.q.out        |    20 +-
 .../vector_partitioned_date_time.q.out          |    12 +-
 .../vector_reduce_groupby_decimal.q.out         |    24 +-
 .../vector_varchar_mapjoin1.q.out               |     1 +
 .../clientpositive/vectorization_0.q.out        |    18 +-
 .../clientpositive/vectorization_13.q.out       |    36 +-
 .../clientpositive/vectorization_15.q.out       |    18 +-
 .../clientpositive/vectorization_16.q.out       |    16 +-
 .../clientpositive/vectorization_17.q.out       |    12 +-
 .../clientpositive/vectorization_7.q.out        |     4 +-
 .../clientpositive/vectorization_8.q.out        |     4 +-
 .../clientpositive/vectorization_9.q.out        |    16 +-
 .../clientpositive/vectorization_limit.q.out    |    16 +-
 .../clientpositive/vectorization_pushdown.q.out |     4 +-
 .../vectorization_short_regress.q.out           |   114 +-
 .../clientpositive/vectorized_case.q.out        |     2 +-
 .../clientpositive/vectorized_casts.q.out       |    66 +-
 .../vectorized_distinct_gby.q.out               |    12 +-
 .../vectorized_nested_mapjoin.q.out             |    26 +-
 .../clientpositive/vectorized_parquet.q.out     |     6 +-
 .../vectorized_parquet_types.q.out              |     6 +-
 .../results/clientpositive/vectorized_ptf.q.out |    67 +-
 .../vectorized_timestamp_funcs.q.out            |    12 +-
 .../results/clientpositive/windowing_udaf.q.out |    12 +
 .../hadoop/hive/serde/test/InnerStruct.java     |     2 +-
 .../hadoop/hive/serde/test/ThriftTestObj.java   |     2 +-
 .../hadoop/hive/serde2/thrift/test/Complex.java |     2 +-
 .../hive/serde2/thrift/test/IntString.java      |     2 +-
 .../hive/serde2/thrift/test/MegaStruct.java     |     2 +-
 .../hive/serde2/thrift/test/MiniStruct.java     |     2 +-
 .../hive/serde2/thrift/test/SetIntString.java   |     2 +-
 .../hive/serde2/ColumnProjectionUtils.java      |    22 +
 .../apache/hadoop/hive/serde2/WriteBuffers.java |    12 +-
 .../serde2/avro/AvroLazyObjectInspector.java    |    19 +-
 .../hadoop/hive/serde2/avro/InstanceCache.java  |    17 +-
 .../hadoop/hive/serde2/lazy/LazyByte.java       |     4 +
 .../hadoop/hive/serde2/lazy/LazyDouble.java     |     4 +
 .../hadoop/hive/serde2/lazy/LazyFloat.java      |     4 +
 .../hadoop/hive/serde2/lazy/LazyInteger.java    |     4 +
 .../hadoop/hive/serde2/lazy/LazyLong.java       |     4 +
 .../hadoop/hive/serde2/lazy/LazyShort.java      |     4 +
 .../hadoop/hive/serde2/lazy/LazyUtils.java      |    28 +
 .../ObjectInspectorConverters.java              |    31 +-
 .../objectinspector/ObjectInspectorFactory.java |    68 +-
 .../objectinspector/ObjectInspectorUtils.java   |    19 +
 .../ReflectionStructObjectInspector.java        |    60 +-
 .../SettableUnionObjectInspector.java           |     4 +-
 .../StandardConstantStructObjectInspector.java  |    51 +
 .../StandardStructObjectInspector.java          |     7 +-
 .../StandardUnionObjectInspector.java           |     4 +-
 .../ThriftUnionObjectInspector.java             |    44 +-
 .../hive/serde2/typeinfo/TypeInfoUtils.java     |     2 +-
 .../avro/TestAvroLazyObjectInspector.java       |    59 +
 .../TestObjectInspectorConverters.java          |    89 +-
 .../TestReflectionObjectInspectors.java         |    71 +-
 .../TestThriftObjectInspectors.java             |    10 +-
 .../hadoop/hive/service/HiveClusterStatus.java  |     2 +-
 .../hive/service/HiveServerException.java       |     2 +-
 .../apache/hadoop/hive/service/ThriftHive.java  |     2 +-
 .../service/cli/thrift/TArrayTypeEntry.java     |     2 +-
 .../hive/service/cli/thrift/TBinaryColumn.java  |     2 +-
 .../hive/service/cli/thrift/TBoolColumn.java    |     2 +-
 .../hive/service/cli/thrift/TBoolValue.java     |     2 +-
 .../hive/service/cli/thrift/TByteColumn.java    |     2 +-
 .../hive/service/cli/thrift/TByteValue.java     |     2 +-
 .../hive/service/cli/thrift/TCLIService.java    |     2 +-
 .../cli/thrift/TCancelDelegationTokenReq.java   |     2 +-
 .../cli/thrift/TCancelDelegationTokenResp.java  |     2 +-
 .../service/cli/thrift/TCancelOperationReq.java |     2 +-
 .../cli/thrift/TCancelOperationResp.java        |     2 +-
 .../service/cli/thrift/TCloseOperationReq.java  |     2 +-
 .../service/cli/thrift/TCloseOperationResp.java |     2 +-
 .../service/cli/thrift/TCloseSessionReq.java    |     2 +-
 .../service/cli/thrift/TCloseSessionResp.java   |     2 +-
 .../hive/service/cli/thrift/TColumnDesc.java    |     2 +-
 .../hive/service/cli/thrift/TDoubleColumn.java  |     2 +-
 .../hive/service/cli/thrift/TDoubleValue.java   |     2 +-
 .../cli/thrift/TExecuteStatementReq.java        |     2 +-
 .../cli/thrift/TExecuteStatementResp.java       |     2 +-
 .../service/cli/thrift/TFetchResultsReq.java    |     2 +-
 .../service/cli/thrift/TFetchResultsResp.java   |     2 +-
 .../service/cli/thrift/TGetCatalogsReq.java     |     2 +-
 .../service/cli/thrift/TGetCatalogsResp.java    |     2 +-
 .../hive/service/cli/thrift/TGetColumnsReq.java |     2 +-
 .../service/cli/thrift/TGetColumnsResp.java     |     2 +-
 .../cli/thrift/TGetDelegationTokenReq.java      |     2 +-
 .../cli/thrift/TGetDelegationTokenResp.java     |     2 +-
 .../service/cli/thrift/TGetFunctionsReq.java    |     2 +-
 .../service/cli/thrift/TGetFunctionsResp.java   |     2 +-
 .../hive/service/cli/thrift/TGetInfoReq.java    |     2 +-
 .../hive/service/cli/thrift/TGetInfoResp.java   |     2 +-
 .../cli/thrift/TGetOperationStatusReq.java      |     2 +-
 .../cli/thrift/TGetOperationStatusResp.java     |     2 +-
 .../cli/thrift/TGetResultSetMetadataReq.java    |     2 +-
 .../cli/thrift/TGetResultSetMetadataResp.java   |     2 +-
 .../hive/service/cli/thrift/TGetSchemasReq.java |     2 +-
 .../service/cli/thrift/TGetSchemasResp.java     |     2 +-
 .../service/cli/thrift/TGetTableTypesReq.java   |     2 +-
 .../service/cli/thrift/TGetTableTypesResp.java  |     2 +-
 .../hive/service/cli/thrift/TGetTablesReq.java  |     2 +-
 .../hive/service/cli/thrift/TGetTablesResp.java |     2 +-
 .../service/cli/thrift/TGetTypeInfoReq.java     |     2 +-
 .../service/cli/thrift/TGetTypeInfoResp.java    |     2 +-
 .../service/cli/thrift/THandleIdentifier.java   |     2 +-
 .../hive/service/cli/thrift/TI16Column.java     |     2 +-
 .../hive/service/cli/thrift/TI16Value.java      |     2 +-
 .../hive/service/cli/thrift/TI32Column.java     |     2 +-
 .../hive/service/cli/thrift/TI32Value.java      |     2 +-
 .../hive/service/cli/thrift/TI64Column.java     |     2 +-
 .../hive/service/cli/thrift/TI64Value.java      |     2 +-
 .../hive/service/cli/thrift/TMapTypeEntry.java  |     2 +-
 .../service/cli/thrift/TOpenSessionReq.java     |     2 +-
 .../service/cli/thrift/TOpenSessionResp.java    |     2 +-
 .../service/cli/thrift/TOperationHandle.java    |     2 +-
 .../service/cli/thrift/TPrimitiveTypeEntry.java |     2 +-
 .../cli/thrift/TRenewDelegationTokenReq.java    |     2 +-
 .../cli/thrift/TRenewDelegationTokenResp.java   |     2 +-
 .../apache/hive/service/cli/thrift/TRow.java    |     2 +-
 .../apache/hive/service/cli/thrift/TRowSet.java |     2 +-
 .../hive/service/cli/thrift/TSessionHandle.java |     2 +-
 .../apache/hive/service/cli/thrift/TStatus.java |     2 +-
 .../hive/service/cli/thrift/TStringColumn.java  |     2 +-
 .../hive/service/cli/thrift/TStringValue.java   |     2 +-
 .../service/cli/thrift/TStructTypeEntry.java    |     2 +-
 .../hive/service/cli/thrift/TTableSchema.java   |     2 +-
 .../hive/service/cli/thrift/TTypeDesc.java      |     2 +-
 .../service/cli/thrift/TTypeQualifiers.java     |     2 +-
 .../service/cli/thrift/TUnionTypeEntry.java     |     2 +-
 .../cli/thrift/TUserDefinedTypeEntry.java       |     2 +-
 .../gen-py/hive_service/ThriftHive-remote       |    21 +-
 .../hive/service/cli/CLIServiceUtils.java       |     7 -
 .../org/apache/hive/service/cli/Column.java     |     2 +-
 .../cli/operation/GetColumnsOperation.java      |    10 +-
 .../cli/operation/GetTablesOperation.java       |     7 +-
 .../cli/operation/LogDivertAppender.java        |   223 +-
 .../service/cli/operation/OperationManager.java |    17 +-
 .../service/cli/session/SessionManager.java     |    42 +-
 .../thrift/RetryingThriftCLIServiceClient.java  |   331 +
 .../apache/hive/service/server/HiveServer2.java |    74 +-
 .../org/apache/hive/service/cli/TestColumn.java |   129 +
 .../cli/TestRetryingThriftCLIServiceClient.java |   133 +
 .../session/TestPluggableHiveSessionImpl.java   |    55 +
 shims/0.20S/pom.xml                             |     8 +-
 .../hadoop/hive/shims/Hadoop20SShims.java       |    46 +-
 shims/0.23/pom.xml                              |    21 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |   145 +-
 shims/common/pom.xml                            |    17 +-
 .../apache/hadoop/hive/shims/HadoopShims.java   |    37 +
 .../hadoop/hive/shims/HadoopShimsSecure.java    |    32 +
 .../hadoop/hive/shims/HiveEventCounter.java     |   102 -
 .../hive/spark/client/SparkClientImpl.java      |     5 +-
 .../src/test/resources/log4j.properties         |    23 -
 spark-client/src/test/resources/log4j2.xml      |    39 +
 storage-api/pom.xml                             |     7 -
 .../hadoop/hive/common/type/HiveDecimal.java    |     1 +
 .../hadoop/hive/ql/io/sarg/PredicateLeaf.java   |     3 +-
 .../hive/ql/io/sarg/SearchArgumentFactory.java  |     5 +-
 .../hive/ql/io/sarg/SearchArgumentImpl.java     |     7 +-
 testutils/ptest2/pom.xml                        |    32 +-
 .../ptest2/src/main/resources/log4j.properties  |    37 -
 testutils/ptest2/src/main/resources/log4j2.xml  |    79 +
 1119 files changed, 67101 insertions(+), 16669 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/70eeadd2/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --cc common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 73610dc,7f29da2..4ce21a3
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@@ -765,8 -765,10 +765,10 @@@ public class HiveConf extends Configura
      HIVEMAPJOINBUCKETCACHESIZE("hive.mapjoin.bucket.cache.size", 100, ""),
  
      HIVEMAPJOINUSEOPTIMIZEDTABLE("hive.mapjoin.optimized.hashtable", true,
 -        "Whether Hive should use memory-optimized hash table for MapJoin. Only works on Tez,\n" +
 -        "because memory-optimized hashtable cannot be serialized."),
 +        "Whether Hive should use memory-optimized hash table for MapJoin.\n" +
 +        "Only works on Tez and Spark, because memory-optimized hashtable cannot be serialized."),
+     HIVEMAPJOINOPTIMIZEDTABLEPROBEPERCENT("hive.mapjoin.optimized.hashtable.probe.percent",
+         (float) 0.5, "Probing space percentage of the optimized hashtable"),
      HIVEUSEHYBRIDGRACEHASHJOIN("hive.mapjoin.hybridgrace.hashtable", true, "Whether to use hybrid" +
          "grace hash join as the join method for mapjoin. Tez only."),
      HIVEHYBRIDGRACEHASHJOINMEMCHECKFREQ("hive.mapjoin.hybridgrace.memcheckfrequency", 1024, "For " +

http://git-wip-us.apache.org/repos/asf/hive/blob/70eeadd2/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/70eeadd2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index 4f66cd6,2e3bd76..1064bd2
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@@ -155,9 -155,7 +155,8 @@@ public class Vectorizer implements Phys
  
    Set<String> supportedAggregationUdfs = new HashSet<String>();
  
-   private PhysicalContext physicalContext = null;
    private HiveConf hiveConf;
 +  private boolean isSpark;
  
    public Vectorizer() {
  


[12/50] [abbrv] hive git commit: HIVE-11751: hive-exec-log4j2.xml settings causes DEBUG messages to be generated and ignored (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Posted by xu...@apache.org.
HIVE-11751: hive-exec-log4j2.xml settings causes DEBUG messages to be generated and ignored (Prasanth Jayachandran reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/753fed62
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/753fed62
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/753fed62

Branch: refs/heads/spark
Commit: 753fed62d639179ef30d8db4d58f44ecfa7d2ee4
Parents: f4361bf
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Fri Sep 11 12:03:12 2015 -0500
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Fri Sep 11 12:03:12 2015 -0500

----------------------------------------------------------------------
 beeline/src/main/resources/beeline-log4j2.xml                   | 5 ++---
 common/src/main/resources/hive-log4j2.xml                       | 5 ++---
 common/src/test/resources/hive-exec-log4j2-test.xml             | 5 ++---
 common/src/test/resources/hive-log4j2-test.xml                  | 5 ++---
 data/conf/hive-log4j2.xml                                       | 5 ++---
 .../test/e2e/templeton/deployers/config/hive/hive-log4j2.xml    | 5 ++---
 hcatalog/webhcat/svr/src/main/config/webhcat-log4j2.xml         | 5 ++---
 ql/src/main/resources/hive-exec-log4j2.xml                      | 5 ++---
 ql/src/main/resources/tez-container-log4j2.xml                  | 5 ++---
 testutils/ptest2/src/main/resources/log4j2.xml                  | 5 ++---
 10 files changed, 20 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/753fed62/beeline/src/main/resources/beeline-log4j2.xml
----------------------------------------------------------------------
diff --git a/beeline/src/main/resources/beeline-log4j2.xml b/beeline/src/main/resources/beeline-log4j2.xml
index 2349c5a..a64f55e 100644
--- a/beeline/src/main/resources/beeline-log4j2.xml
+++ b/beeline/src/main/resources/beeline-log4j2.xml
@@ -20,7 +20,6 @@
  packages="org.apache.hadoop.hive.ql.log">
 
   <Properties>
-    <Property name="hive.log.threshold">DEBUG</Property>
     <Property name="hive.log.level">WARN</Property>
     <Property name="hive.root.logger">console</Property>
   </Properties>
@@ -32,8 +31,8 @@
   </Appenders>
 
   <Loggers>
-    <Root level="${sys:hive.log.threshold}">
-      <AppenderRef ref="${sys:hive.root.logger}" level="${sys:hive.log.level}"/>
+    <Root level="${sys:hive.log.level}">
+      <AppenderRef ref="${sys:hive.root.logger}"/>
     </Root>
   </Loggers>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/753fed62/common/src/main/resources/hive-log4j2.xml
----------------------------------------------------------------------
diff --git a/common/src/main/resources/hive-log4j2.xml b/common/src/main/resources/hive-log4j2.xml
index 31b8fcc..3834547 100644
--- a/common/src/main/resources/hive-log4j2.xml
+++ b/common/src/main/resources/hive-log4j2.xml
@@ -20,7 +20,6 @@
  packages="org.apache.hadoop.hive.ql.log">
 
   <Properties>
-    <Property name="hive.log.threshold">ALL</Property>
     <Property name="hive.log.level">INFO</Property>
     <Property name="hive.root.logger">DRFA</Property>
     <Property name="hive.log.dir">${sys:java.io.tmpdir}/${sys:user.name}</Property>
@@ -68,8 +67,8 @@
   </Appenders>
 
   <Loggers>
-    <Root level="${sys:hive.log.threshold}">
-      <AppenderRef ref="${sys:hive.root.logger}" level="${sys:hive.log.level}"/>
+    <Root level="${sys:hive.log.level}">
+      <AppenderRef ref="${sys:hive.root.logger}"/>
       <AppenderRef ref="EventCounter" />
     </Root>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/753fed62/common/src/test/resources/hive-exec-log4j2-test.xml
----------------------------------------------------------------------
diff --git a/common/src/test/resources/hive-exec-log4j2-test.xml b/common/src/test/resources/hive-exec-log4j2-test.xml
index 1d91b26..03487de 100644
--- a/common/src/test/resources/hive-exec-log4j2-test.xml
+++ b/common/src/test/resources/hive-exec-log4j2-test.xml
@@ -20,7 +20,6 @@
  packages="org.apache.hadoop.hive.ql.log">
 
   <Properties>
-    <Property name="hive.log.threshold">DEBUG</Property>
     <Property name="hive.log.level">INFO</Property>
     <Property name="hive.root.logger">FA</Property>
     <Property name="hive.log.dir">${sys:test.tmp.dir}/${sys:user.name}-TestHiveLogging</Property>
@@ -41,8 +40,8 @@
   </Appenders>
 
   <Loggers>
-    <Root level="${sys:hive.log.threshold}">
-      <AppenderRef ref="${sys:hive.root.logger}" level="${sys:hive.log.level}"/>
+    <Root level="${sys:hive.log.level}">
+      <AppenderRef ref="${sys:hive.root.logger}"/>
       <AppenderRef ref="EventCounter" />
     </Root>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/753fed62/common/src/test/resources/hive-log4j2-test.xml
----------------------------------------------------------------------
diff --git a/common/src/test/resources/hive-log4j2-test.xml b/common/src/test/resources/hive-log4j2-test.xml
index 98ca6f8..0297e88 100644
--- a/common/src/test/resources/hive-log4j2-test.xml
+++ b/common/src/test/resources/hive-log4j2-test.xml
@@ -20,7 +20,6 @@
  packages="org.apache.hadoop.hive.ql.log">
 
   <Properties>
-    <Property name="hive.log.threshold">DEBUG</Property>
     <Property name="hive.log.level">WARN</Property>
     <Property name="hive.root.logger">DRFA</Property>
     <Property name="hive.log.dir">${sys:test.tmp.dir}/${sys:user.name}-TestHiveLogging</Property>
@@ -50,8 +49,8 @@
   </Appenders>
 
   <Loggers>
-    <Root level="${sys:hive.log.threshold}">
-      <AppenderRef ref="${sys:hive.root.logger}" level="${sys:hive.log.level}"/>
+    <Root level="${sys:hive.log.level}">
+      <AppenderRef ref="${sys:hive.root.logger}"/>
       <AppenderRef ref="EventCounter" />
     </Root>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/753fed62/data/conf/hive-log4j2.xml
----------------------------------------------------------------------
diff --git a/data/conf/hive-log4j2.xml b/data/conf/hive-log4j2.xml
index 452f01f..48a584f 100644
--- a/data/conf/hive-log4j2.xml
+++ b/data/conf/hive-log4j2.xml
@@ -20,7 +20,6 @@
  packages="org.apache.hadoop.hive.ql.log">
 
   <Properties>
-    <Property name="hive.log.threshold">DEBUG</Property>
     <Property name="hive.log.level">DEBUG</Property>
     <Property name="hive.root.logger">DRFA</Property>
     <Property name="hive.log.dir">${sys:test.tmp.dir}/log</Property>
@@ -68,8 +67,8 @@
   </Appenders>
 
   <Loggers>
-    <Root level="${sys:hive.log.threshold}">
-      <AppenderRef ref="${sys:hive.root.logger}" level="${sys:hive.log.level}"/>
+    <Root level="${sys:hive.log.level}">
+      <AppenderRef ref="${sys:hive.root.logger}"/>
       <AppenderRef ref="EventCounter" />
     </Root>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/753fed62/hcatalog/src/test/e2e/templeton/deployers/config/hive/hive-log4j2.xml
----------------------------------------------------------------------
diff --git a/hcatalog/src/test/e2e/templeton/deployers/config/hive/hive-log4j2.xml b/hcatalog/src/test/e2e/templeton/deployers/config/hive/hive-log4j2.xml
index 30f7603..87e18e2 100644
--- a/hcatalog/src/test/e2e/templeton/deployers/config/hive/hive-log4j2.xml
+++ b/hcatalog/src/test/e2e/templeton/deployers/config/hive/hive-log4j2.xml
@@ -20,7 +20,6 @@
  packages="org.apache.hadoop.hive.ql.log">
 
   <Properties>
-    <Property name="hive.log.threshold">ALL</Property>
     <Property name="hive.log.level">DEBUG</Property>
     <Property name="hive.root.logger">DRFA</Property>
     <Property name="hive.log.dir">${sys:java.io.tmpdir}/${sys:user.name}</Property>
@@ -68,8 +67,8 @@
   </Appenders>
 
   <Loggers>
-    <Root level="${sys:hive.log.threshold}">
-      <AppenderRef ref="${sys:hive.root.logger}" level="${sys:hive.log.level}"/>
+    <Root level="${sys:hive.log.level}">
+      <AppenderRef ref="${sys:hive.root.logger}"/>
       <AppenderRef ref="EventCounter" />
     </Root>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/753fed62/hcatalog/webhcat/svr/src/main/config/webhcat-log4j2.xml
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/svr/src/main/config/webhcat-log4j2.xml b/hcatalog/webhcat/svr/src/main/config/webhcat-log4j2.xml
index 96f0974..ef8e73d 100644
--- a/hcatalog/webhcat/svr/src/main/config/webhcat-log4j2.xml
+++ b/hcatalog/webhcat/svr/src/main/config/webhcat-log4j2.xml
@@ -20,7 +20,6 @@
  packages="org.apache.hadoop.hive.ql.log">
 
   <Properties>
-    <Property name="webhcat.log.threshold">DEBUG</Property>
     <Property name="webhcat.log.level">INFO</Property>
     <Property name="webhcat.root.logger">standard</Property>
     <Property name="webhcat.log.dir">.</Property>
@@ -48,8 +47,8 @@
   </Appenders>
 
   <Loggers>
-    <Root level="${sys:webhcat.log.threshold}">
-      <AppenderRef ref="${sys:webhcat.root.logger}" level="${sys:webhcat.log.level}"/>
+    <Root level="${sys:webhcat.log.level}">
+      <AppenderRef ref="${sys:webhcat.root.logger}"/>
     </Root>
 
     <Logger name="com.sun.jersey" level="DEBUG">

http://git-wip-us.apache.org/repos/asf/hive/blob/753fed62/ql/src/main/resources/hive-exec-log4j2.xml
----------------------------------------------------------------------
diff --git a/ql/src/main/resources/hive-exec-log4j2.xml b/ql/src/main/resources/hive-exec-log4j2.xml
index 8b520a2..8ed8b60 100644
--- a/ql/src/main/resources/hive-exec-log4j2.xml
+++ b/ql/src/main/resources/hive-exec-log4j2.xml
@@ -20,7 +20,6 @@
  packages="org.apache.hadoop.hive.ql.log">
 
   <Properties>
-    <Property name="hive.log.threshold">DEBUG</Property>
     <Property name="hive.log.level">INFO</Property>
     <Property name="hive.root.logger">FA</Property>
     <Property name="hive.log.dir">${sys:java.io.tmpdir}/${sys:user.name}</Property>
@@ -67,8 +66,8 @@
   </Appenders>
 
   <Loggers>
-    <Root level="${sys:hive.log.threshold}">
-      <AppenderRef ref="${sys:hive.root.logger}" level="${sys:hive.log.level}"/>
+    <Root level="${sys:hive.log.level}">
+      <AppenderRef ref="${sys:hive.root.logger}"/>
       <AppenderRef ref="EventCounter" />
     </Root>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/753fed62/ql/src/main/resources/tez-container-log4j2.xml
----------------------------------------------------------------------
diff --git a/ql/src/main/resources/tez-container-log4j2.xml b/ql/src/main/resources/tez-container-log4j2.xml
index be949dc..604e586 100644
--- a/ql/src/main/resources/tez-container-log4j2.xml
+++ b/ql/src/main/resources/tez-container-log4j2.xml
@@ -20,7 +20,6 @@
  packages="org.apache.hadoop.hive.ql.log">
 
   <Properties>
-    <Property name="tez.container.log.threshold">ALL</Property>
     <Property name="tez.container.log.level">INFO</Property>
     <Property name="tez.container.root.logger">CLA</Property>
     <Property name="tez.container.log.dir">${sys:yarn.app.container.log.dir}</Property>
@@ -41,8 +40,8 @@
   </Appenders>
 
   <Loggers>
-    <Root level="${sys:tez.container.log.threshold}">
-      <AppenderRef ref="${sys:tez.container.root.logger}" level="${sys:tez.container.log.level}"/>
+    <Root level="${sys:tez.container.log.level}">
+      <AppenderRef ref="${sys:tez.container.root.logger}"/>
     </Root>
   </Loggers>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/753fed62/testutils/ptest2/src/main/resources/log4j2.xml
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/resources/log4j2.xml b/testutils/ptest2/src/main/resources/log4j2.xml
index 42141b7..8eb3234 100644
--- a/testutils/ptest2/src/main/resources/log4j2.xml
+++ b/testutils/ptest2/src/main/resources/log4j2.xml
@@ -20,7 +20,6 @@
  packages="org.apache.hadoop.hive.ql.log">
 
   <Properties>
-    <Property name="hive.ptest.log.threshold">DEBUG</Property>
     <Property name="hive.ptest.log.level">DEBUG</Property>
     <Property name="hive.ptest.root.logger">FILE</Property>
     <Property name="hive.ptest.log.dir">target</Property>
@@ -45,8 +44,8 @@
   </Appenders>
 
   <Loggers>
-    <Root level="${sys:hive.ptest.log.threshold}">
-      <AppenderRef ref="${sys:hive.ptest.root.logger}" level="${sys:hive.ptest.log.level}"/>
+    <Root level="${sys:hive.ptest.log.level}">
+      <AppenderRef ref="${sys:hive.ptest.root.logger}"/>
     </Root>
 
     <Logger name="org.apache.http" level="INFO">


[11/50] [abbrv] hive git commit: HIVE-11645 : Add in-place updates for dynamic partitions loading (Ashutosh Chauhan via Prasanth J)

Posted by xu...@apache.org.
HIVE-11645 : Add in-place updates for dynamic partitions loading (Ashutosh Chauhan via Prasanth J)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f4361bf3
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f4361bf3
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f4361bf3

Branch: refs/heads/spark
Commit: f4361bf30689c4767e966e11c610f7ead632415a
Parents: 9fe8802
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Thu Sep 10 14:52:43 2015 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Thu Sep 10 14:52:43 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/MoveTask.java    | 12 ++--
 .../apache/hadoop/hive/ql/exec/StatsTask.java   | 13 +++-
 .../hadoop/hive/ql/exec/tez/InPlaceUpdates.java | 65 +++++++++++++++++++
 .../hadoop/hive/ql/exec/tez/TezJobMonitor.java  | 66 ++------------------
 .../apache/hadoop/hive/ql/metadata/Hive.java    | 25 ++++++--
 5 files changed, 106 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f4361bf3/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index 50c4a96..a1f8973 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -361,7 +361,7 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
             if (dps != null && dps.size() > 0) {
               pushFeed(FeedType.DYNAMIC_PARTITIONS, dps);
             }
-
+            console.printInfo(System.getProperty("line.separator"));
             long startTime = System.currentTimeMillis();
             // load the list of DP partitions and return the list of partition specs
             // TODO: In a follow-up to HIVE-1361, we should refactor loadDynamicPartitions
@@ -381,8 +381,9 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
                 isSkewedStoredAsDirs(tbd),
                 work.getLoadTableWork().getWriteType() != AcidUtils.Operation.NOT_ACID,
                 SessionState.get().getTxnMgr().getCurrentTxnId());
-            console.printInfo("\t Time taken for load dynamic partitions : "  +
-                (System.currentTimeMillis() - startTime));
+
+            console.printInfo("\t Time taken to load dynamic partitions: "  +
+                (System.currentTimeMillis() - startTime)/1000.0 + " seconds");
 
             if (dp.size() == 0 && conf.getBoolVar(HiveConf.ConfVars.HIVE_ERROR_ON_EMPTY_PARTITION)) {
               throw new HiveException("This query creates no partitions." +
@@ -425,11 +426,10 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
                 SessionState.get().getLineageState().setLineage(tbd.getSourcePath(), dc,
                     table.getCols());
               }
-
-              console.printInfo("\tLoading partition " + entry.getKey());
+              LOG.info("\tLoading partition " + entry.getKey());
             }
             console.printInfo("\t Time taken for adding to write entity : " +
-                (System.currentTimeMillis() - startTime));
+                (System.currentTimeMillis() - startTime)/1000.0 + " seconds");
             dc = null; // reset data container to prevent it being added again.
           } else { // static partitions
             List<String> partVals = MetaStoreUtils.getPvals(table.getPartCols(),

http://git-wip-us.apache.org/repos/asf/hive/blob/f4361bf3/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
index 2a8167a..41ece04 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
@@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -182,8 +183,10 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
         parameters.put(StatsSetupConst.STATS_GENERATED_VIA_STATS_TASK, StatsSetupConst.TRUE);
 
         db.alterTable(tableFullName, new Table(tTable));
-
-        console.printInfo("Table " + tableFullName + " stats: [" + toString(parameters) + ']');
+        if (conf.getBoolVar(ConfVars.TEZ_EXEC_SUMMARY)) {
+          console.printInfo("Table " + tableFullName + " stats: [" + toString(parameters) + ']');
+        }
+        LOG.info("Table " + tableFullName + " stats: [" + toString(parameters) + ']');
       } else {
         // Partitioned table:
         // Need to get the old stats of the partition
@@ -215,7 +218,11 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
           parameters.put(StatsSetupConst.STATS_GENERATED_VIA_STATS_TASK, StatsSetupConst.TRUE);
           updates.add(new Partition(table, tPart));
 
-          console.printInfo("Partition " + tableFullName + partn.getSpec() +
+          if (conf.getBoolVar(ConfVars.TEZ_EXEC_SUMMARY)) {
+            console.printInfo("Partition " + tableFullName + partn.getSpec() +
+            " stats: [" + toString(parameters) + ']');
+          }
+          LOG.info("Partition " + tableFullName + partn.getSpec() +
               " stats: [" + toString(parameters) + ']');
         }
         if (!updates.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f4361bf3/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/InPlaceUpdates.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/InPlaceUpdates.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/InPlaceUpdates.java
new file mode 100644
index 0000000..6ecfe71
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/InPlaceUpdates.java
@@ -0,0 +1,65 @@
+package org.apache.hadoop.hive.ql.exec.tez;
+
+import static org.fusesource.jansi.Ansi.ansi;
+import static org.fusesource.jansi.internal.CLibrary.STDERR_FILENO;
+import static org.fusesource.jansi.internal.CLibrary.STDOUT_FILENO;
+import static org.fusesource.jansi.internal.CLibrary.isatty;
+
+import java.io.PrintStream;
+
+import jline.TerminalFactory;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.fusesource.jansi.Ansi;
+
+public class InPlaceUpdates {
+
+  private static final int MIN_TERMINAL_WIDTH = 80;
+
+  static boolean isUnixTerminal() {
+
+    String os = System.getProperty("os.name");
+    if (os.startsWith("Windows")) {
+      // we do not support Windows, we will revisit this if we really need it for windows.
+      return false;
+    }
+
+    // We must be on some unix variant..
+    // check if standard out is a terminal
+    try {
+      // isatty system call will return 1 if the file descriptor is terminal else 0
+      if (isatty(STDOUT_FILENO) == 0) {
+        return false;
+      }
+      if (isatty(STDERR_FILENO) == 0) {
+        return false;
+      }
+    } catch (NoClassDefFoundError ignore) {
+      // These errors happen if the JNI lib is not available for your platform.
+      return false;
+    } catch (UnsatisfiedLinkError ignore) {
+      // These errors happen if the JNI lib is not available for your platform.
+      return false;
+    }
+    return true;
+  }
+
+  public static boolean inPlaceEligible(HiveConf conf) {
+    boolean inPlaceUpdates = HiveConf.getBoolVar(conf, HiveConf.ConfVars.TEZ_EXEC_INPLACE_PROGRESS);
+
+    // we need at least 80 chars wide terminal to display in-place updates properly
+    return inPlaceUpdates && !SessionState.getConsole().getIsSilent() && isUnixTerminal()
+      && TerminalFactory.get().getWidth() >= MIN_TERMINAL_WIDTH;
+  }
+
+  public static void reprintLine(PrintStream out, String line) {
+    out.print(ansi().eraseLine(Ansi.Erase.ALL).a(line).a('\n').toString());
+    out.flush();
+  }
+
+  public static void rePositionCursor(PrintStream ps) {
+    ps.print(ansi().cursorUp(0).toString());
+    ps.flush();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f4361bf3/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java
index 1a4decf..1e1603b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java
@@ -20,9 +20,6 @@ package org.apache.hadoop.hive.ql.exec.tez;
 
 import static org.apache.tez.dag.api.client.DAGStatus.State.RUNNING;
 import static org.fusesource.jansi.Ansi.ansi;
-import static org.fusesource.jansi.internal.CLibrary.STDOUT_FILENO;
-import static org.fusesource.jansi.internal.CLibrary.STDERR_FILENO;
-import static org.fusesource.jansi.internal.CLibrary.isatty;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
@@ -74,7 +71,7 @@ import jline.TerminalFactory;
 public class TezJobMonitor {
 
   private static final String CLASS_NAME = TezJobMonitor.class.getName();
-  private static final int MIN_TERMINAL_WIDTH = 80;
+
   private static final int COLUMN_1_WIDTH = 16;
   private static final int SEPARATOR_WIDTH = 80;
 
@@ -156,42 +153,13 @@ public class TezJobMonitor {
     }
   }
 
-  private static boolean isUnixTerminal() {
-
-    String os = System.getProperty("os.name");
-    if (os.startsWith("Windows")) {
-      // we do not support Windows, we will revisit this if we really need it for windows.
-      return false;
-    }
-
-    // We must be on some unix variant..
-    // check if standard out is a terminal
-    try {
-      // isatty system call will return 1 if the file descriptor is terminal else 0
-      if (isatty(STDOUT_FILENO) == 0) {
-        return false;
-      }
-      if (isatty(STDERR_FILENO) == 0) {
-        return false;
-      }
-    } catch (NoClassDefFoundError ignore) {
-      // These errors happen if the JNI lib is not available for your platform.
-      return false;
-    } catch (UnsatisfiedLinkError ignore) {
-      // These errors happen if the JNI lib is not available for your platform.
-      return false;
-    }
-    return true;
-  }
-
   /**
    * NOTE: Use this method only if isUnixTerminal is true.
    * Erases the current line and prints the given line.
    * @param line - line to print
    */
   public void reprintLine(String line) {
-    out.print(ansi().eraseLine(Ansi.Erase.ALL).a(line).a('\n').toString());
-    out.flush();
+    InPlaceUpdates.reprintLine(out, line);
     lines++;
   }
 
@@ -234,15 +202,6 @@ public class TezJobMonitor {
   }
 
   /**
-   * NOTE: Use this method only if isUnixTerminal is true.
-   * Gets the width of the terminal
-   * @return - width of terminal
-   */
-  public int getTerminalWidth() {
-    return TerminalFactory.get().getWidth();
-  }
-
-  /**
    * monitorExecution handles status printing, failures during execution and final status retrieval.
    *
    * @param dagClient client that was used to kick off the job
@@ -265,26 +224,11 @@ public class TezJobMonitor {
     Set<StatusGetOpts> opts = new HashSet<StatusGetOpts>();
     Heartbeater heartbeater = new Heartbeater(txnMgr, conf);
     long startTime = 0;
-    boolean isProfileEnabled = conf.getBoolVar(conf, HiveConf.ConfVars.TEZ_EXEC_SUMMARY) ||
+    boolean isProfileEnabled = HiveConf.getBoolVar(conf, HiveConf.ConfVars.TEZ_EXEC_SUMMARY) ||
       Utilities.isPerfOrAboveLogging(conf);
-    boolean inPlaceUpdates = conf.getBoolVar(conf, HiveConf.ConfVars.TEZ_EXEC_INPLACE_PROGRESS);
-    boolean wideTerminal = false;
-    boolean isTerminal = inPlaceUpdates == true ? isUnixTerminal() : false;
-
-    // we need at least 80 chars wide terminal to display in-place updates properly
-    if (isTerminal) {
-      if (getTerminalWidth() >= MIN_TERMINAL_WIDTH) {
-        wideTerminal = true;
-      }
-    }
-
-    boolean inPlaceEligible = false;
-    if (inPlaceUpdates && isTerminal && wideTerminal && !console.getIsSilent()) {
-      inPlaceEligible = true;
-    }
 
+    boolean inPlaceEligible = InPlaceUpdates.inPlaceEligible(conf);
     shutdownList.add(dagClient);
-
     console.printInfo("\n");
     perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_DAG);
     perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_SUBMIT_TO_RUNNING);
@@ -470,7 +414,7 @@ public class TezJobMonitor {
       DAGClient dagClient, HiveConf conf, DAG dag) {
 
     /* Strings for headers and counters */
-    String hiveCountersGroup = conf.getVar(conf, HiveConf.ConfVars.HIVECOUNTERGROUP);
+    String hiveCountersGroup = HiveConf.getVar(conf, HiveConf.ConfVars.HIVECOUNTERGROUP);
     Set<StatusGetOpts> statusGetOpts = EnumSet.of(StatusGetOpts.GET_COUNTERS);
     TezCounters hiveCounters = null;
     try {

http://git-wip-us.apache.org/repos/asf/hive/blob/f4361bf3/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index c449aee..c78e8f4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.metadata;
 
 import com.google.common.collect.Sets;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -81,6 +82,7 @@ import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.FunctionTask;
 import org.apache.hadoop.hive.ql.exec.FunctionUtils;
 import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.tez.InPlaceUpdates;
 import org.apache.hadoop.hive.ql.index.HiveIndexHandler;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.optimizer.listbucketingpruner.ListBucketingPrunerUtils;
@@ -101,6 +103,7 @@ import org.apache.thrift.TException;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -124,7 +127,6 @@ import static org.apache.hadoop.hive.serde.serdeConstants.MAPKEY_DELIM;
 import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT;
 import static org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME;
 
-
 /**
  * This class has functions that implement meta data/DDL operations using calls
  * to the metastore.
@@ -1606,22 +1608,31 @@ private void constructOneLBLocationMap(FileStatus fSta,
         }
       }
 
-      if (validPartitions.size() == 0) {
+      int partsToLoad = validPartitions.size();
+      if (partsToLoad == 0) {
         LOG.warn("No partition is generated by dynamic partitioning");
       }
 
-      if (validPartitions.size() > conf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS)) {
-        throw new HiveException("Number of dynamic partitions created is " + validPartitions.size()
+      if (partsToLoad > conf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS)) {
+        throw new HiveException("Number of dynamic partitions created is " + partsToLoad
             + ", which is more than "
             + conf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS)
             +". To solve this try to set " + HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.varname
-            + " to at least " + validPartitions.size() + '.');
+            + " to at least " + partsToLoad + '.');
       }
 
       Table tbl = getTable(tableName);
       // for each dynamically created DP directory, construct a full partition spec
       // and load the partition based on that
       Iterator<Path> iter = validPartitions.iterator();
+      LOG.info("Going to load " + partsToLoad + " partitions.");
+      PrintStream ps = null;
+      boolean inPlaceEligible = conf.getLong("fs.trash.interval", 0) <= 0
+          && InPlaceUpdates.inPlaceEligible(conf);
+      if(inPlaceEligible) {
+        ps = SessionState.getConsole().getInfoStream();
+      }
+      int partitionsLoaded = 0;
       while (iter.hasNext()) {
         // get the dynamically created directory
         Path partPath = iter.next();
@@ -1634,6 +1645,10 @@ private void constructOneLBLocationMap(FileStatus fSta,
         Partition newPartition = loadPartition(partPath, tbl, fullPartSpec, replace,
             holdDDLTime, true, listBucketingEnabled, false, isAcid);
         partitionsMap.put(fullPartSpec, newPartition);
+        if (inPlaceEligible) {
+          InPlaceUpdates.rePositionCursor(ps);
+          InPlaceUpdates.reprintLine(ps, "Loaded : " + ++partitionsLoaded + "/" + partsToLoad +" partitions.");
+        }
         LOG.info("New loading path = " + partPath + " with partSpec " + fullPartSpec);
       }
       if (isAcid) {


[38/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_orderby_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_orderby_5.q.out b/ql/src/test/results/clientpositive/tez/vector_orderby_5.q.out
index af15925..3712b16 100644
--- a/ql/src/test/results/clientpositive/tez/vector_orderby_5.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_orderby_5.q.out
@@ -126,11 +126,11 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: bo (type: boolean), b (type: bigint)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: bo, b
                     Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col1)
-                      keys: _col0 (type: boolean)
+                      aggregations: max(b)
+                      keys: bo (type: boolean)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_outer_join2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_outer_join2.q.out b/ql/src/test/results/clientpositive/tez/vector_outer_join2.q.out
index f9159eb..2315cb9 100644
--- a/ql/src/test/results/clientpositive/tez/vector_outer_join2.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_outer_join2.q.out
@@ -227,19 +227,15 @@ STAGE PLANS:
                           1 Map 4
                         Statistics: Num rows: 24 Data size: 5361 Basic stats: COMPLETE Column stats: NONE
                         HybridGraceHashJoin: true
-                        Select Operator
-                          expressions: _col1 (type: bigint)
-                          outputColumnNames: _col0
-                          Statistics: Num rows: 24 Data size: 5361 Basic stats: COMPLETE Column stats: NONE
-                          Group By Operator
-                            aggregations: count(), sum(_col0)
-                            mode: hash
-                            outputColumnNames: _col0, _col1
+                        Group By Operator
+                          aggregations: count(), sum(_col1)
+                          mode: hash
+                          outputColumnNames: _col0, _col1
+                          Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
+                          Reduce Output Operator
+                            sort order: 
                             Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-                            Reduce Output Operator
-                              sort order: 
-                              Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-                              value expressions: _col0 (type: bigint), _col1 (type: bigint)
+                            value expressions: _col0 (type: bigint), _col1 (type: bigint)
             Execution mode: vectorized
         Map 3 
             Map Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_partition_diff_num_cols.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_partition_diff_num_cols.q.out b/ql/src/test/results/clientpositive/tez/vector_partition_diff_num_cols.q.out
index 65b2ff1..c9ceeb4 100644
--- a/ql/src/test/results/clientpositive/tez/vector_partition_diff_num_cols.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_partition_diff_num_cols.q.out
@@ -100,10 +100,10 @@ STAGE PLANS:
                   Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: inv_quantity_on_hand (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: inv_quantity_on_hand
                     Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(inv_quantity_on_hand)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -226,10 +226,10 @@ STAGE PLANS:
                   Statistics: Num rows: 200 Data size: 11876 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: inv_quantity_on_hand (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: inv_quantity_on_hand
                     Statistics: Num rows: 200 Data size: 11876 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(inv_quantity_on_hand)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -353,10 +353,10 @@ STAGE PLANS:
                   Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: inv_quantity_on_hand (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: inv_quantity_on_hand
                     Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(inv_quantity_on_hand)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -461,10 +461,10 @@ STAGE PLANS:
                   Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: inv_quantity_on_hand (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: inv_quantity_on_hand
                     Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(inv_quantity_on_hand)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -575,10 +575,10 @@ STAGE PLANS:
                   Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: inv_quantity_on_hand (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: inv_quantity_on_hand
                     Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(inv_quantity_on_hand)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_partitioned_date_time.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_partitioned_date_time.q.out b/ql/src/test/results/clientpositive/tez/vector_partitioned_date_time.q.out
index 4b45ac1..70d415d 100644
--- a/ql/src/test/results/clientpositive/tez/vector_partitioned_date_time.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_partitioned_date_time.q.out
@@ -379,11 +379,11 @@ STAGE PLANS:
                   Statistics: Num rows: 137 Data size: 39456 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: fl_date (type: date)
-                    outputColumnNames: _col0
+                    outputColumnNames: fl_date
                     Statistics: Num rows: 137 Data size: 39456 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: date)
+                      keys: fl_date (type: date)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 137 Data size: 39456 Basic stats: COMPLETE Column stats: NONE
@@ -1159,11 +1159,11 @@ STAGE PLANS:
                   Statistics: Num rows: 137 Data size: 31776 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: fl_date (type: date)
-                    outputColumnNames: _col0
+                    outputColumnNames: fl_date
                     Statistics: Num rows: 137 Data size: 31776 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: date)
+                      keys: fl_date (type: date)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 137 Data size: 31776 Basic stats: COMPLETE Column stats: NONE
@@ -1963,11 +1963,11 @@ STAGE PLANS:
                   Statistics: Num rows: 137 Data size: 33968 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: fl_time (type: timestamp)
-                    outputColumnNames: _col0
+                    outputColumnNames: fl_time
                     Statistics: Num rows: 137 Data size: 33968 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: timestamp)
+                      keys: fl_time (type: timestamp)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 137 Data size: 33968 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vector_reduce_groupby_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_reduce_groupby_decimal.q.out b/ql/src/test/results/clientpositive/tez/vector_reduce_groupby_decimal.q.out
index 528c2cb..14aa777 100644
--- a/ql/src/test/results/clientpositive/tez/vector_reduce_groupby_decimal.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_reduce_groupby_decimal.q.out
@@ -44,22 +44,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (cdecimal1 is not null and cdecimal2 is not null) (type: boolean)
                     Statistics: Num rows: 1526 Data size: 360136 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: cint (type: int), cdouble (type: double), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14))
-                      outputColumnNames: _col0, _col1, _col2, _col3
+                    Group By Operator
+                      aggregations: min(cdecimal1)
+                      keys: cint (type: int), cdouble (type: double), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14))
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1526 Data size: 360136 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: min(_col2)
-                        keys: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14))
-                        mode: hash
-                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14))
+                        sort order: ++++
+                        Map-reduce partition columns: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14))
                         Statistics: Num rows: 1526 Data size: 360136 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14))
-                          sort order: ++++
-                          Map-reduce partition columns: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14))
-                          Statistics: Num rows: 1526 Data size: 360136 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col4 (type: decimal(20,10))
+                        value expressions: _col4 (type: decimal(20,10))
             Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorization_0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorization_0.q.out b/ql/src/test/results/clientpositive/tez/vectorization_0.q.out
index 18e042d..4580fe0 100644
--- a/ql/src/test/results/clientpositive/tez/vectorization_0.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorization_0.q.out
@@ -133,10 +133,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctinyint
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(ctinyint)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -439,10 +439,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cbigint (type: bigint)
-                    outputColumnNames: _col0
+                    outputColumnNames: cbigint
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(cbigint)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -745,10 +745,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cfloat (type: float)
-                    outputColumnNames: _col0
+                    outputColumnNames: cfloat
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(cfloat)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -998,10 +998,10 @@ STAGE PLANS:
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cbigint (type: bigint), cfloat (type: float), ctinyint (type: tinyint)
-                      outputColumnNames: _col0, _col1, _col2
+                      outputColumnNames: cbigint, cfloat, ctinyint
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: avg(_col0), stddev_pop(_col0), var_samp(_col0), count(), sum(_col1), min(_col2)
+                        aggregations: avg(cbigint), stddev_pop(cbigint), var_samp(cbigint), count(), sum(cfloat), min(ctinyint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                         Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorization_13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorization_13.q.out b/ql/src/test/results/clientpositive/tez/vectorization_13.q.out
index 6214640..d153dd9 100644
--- a/ql/src/test/results/clientpositive/tez/vectorization_13.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorization_13.q.out
@@ -89,19 +89,19 @@ STAGE PLANS:
                     predicate: (((cfloat < 3569.0) and (10.175 >= cdouble) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > 11.0) and (UDFToDouble(ctimestamp2) <> 12.0) and (UDFToDouble(ctinyint) < 9763215.5639))) (type: boolean)
                     Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      expressions: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
+                      outputColumnNames: ctinyint, cfloat, cstring1, ctimestamp1, cboolean1
                       Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: max(_col1), sum(_col3), stddev_pop(_col3), stddev_pop(_col1), max(_col3), min(_col1)
-                        keys: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                        aggregations: max(ctinyint), sum(cfloat), stddev_pop(cfloat), stddev_pop(ctinyint), max(cfloat), min(ctinyint)
+                        keys: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                         Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                          key expressions: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                           sort order: +++++
-                          Map-reduce partition columns: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                          Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                           Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: struct<count:bigint,sum:double,variance:double>), _col9 (type: float), _col10 (type: tinyint)
             Execution mode: vectorized
@@ -109,12 +109,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: max(VALUE._col0), sum(VALUE._col1), stddev_pop(VALUE._col2), stddev_pop(VALUE._col3), max(VALUE._col4), min(VALUE._col5)
-                keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: float), KEY._col2 (type: string), KEY._col3 (type: timestamp), KEY._col4 (type: boolean)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                 Statistics: Num rows: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col3)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col1) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col1) + _col5))) / UDFToDouble(_col1)) (type: double), _col10 (type: tinyint)
+                  expressions: _col4 (type: boolean), _col0 (type: tinyint), _col3 (type: timestamp), _col1 (type: float), _col2 (type: string), (- _col0) (type: tinyint), _col5 (type: tinyint), ((- _col0) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col0) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col1)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col0) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col0) + _col5))) / UDFToDouble(_col0)) (type: double), _col10 (type: tinyint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
                   Statistics: Num rows: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
@@ -342,19 +342,19 @@ STAGE PLANS:
                     predicate: (((cfloat < 3569.0) and (10.175 >= cdouble) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -1.388) and (UDFToDouble(ctimestamp2) <> -1.3359999999999999) and (UDFToDouble(ctinyint) < 9763215.5639))) (type: boolean)
                     Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      expressions: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
+                      outputColumnNames: ctinyint, cfloat, cstring1, ctimestamp1, cboolean1
                       Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: max(_col1), sum(_col3), stddev_pop(_col3), stddev_pop(_col1), max(_col3), min(_col1)
-                        keys: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                        aggregations: max(ctinyint), sum(cfloat), stddev_pop(cfloat), stddev_pop(ctinyint), max(cfloat), min(ctinyint)
+                        keys: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                         Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                          key expressions: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                           sort order: +++++
-                          Map-reduce partition columns: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                          Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                           Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: struct<count:bigint,sum:double,variance:double>), _col9 (type: float), _col10 (type: tinyint)
             Execution mode: vectorized
@@ -362,12 +362,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: max(VALUE._col0), sum(VALUE._col1), stddev_pop(VALUE._col2), stddev_pop(VALUE._col3), max(VALUE._col4), min(VALUE._col5)
-                keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: float), KEY._col2 (type: string), KEY._col3 (type: timestamp), KEY._col4 (type: boolean)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                 Statistics: Num rows: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col3)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col1) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col1) + _col5))) / UDFToDouble(_col1)) (type: double), _col10 (type: tinyint)
+                  expressions: _col4 (type: boolean), _col0 (type: tinyint), _col3 (type: timestamp), _col1 (type: float), _col2 (type: string), (- _col0) (type: tinyint), _col5 (type: tinyint), ((- _col0) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col0) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col1)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col0) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col0) + _col5))) / UDFToDouble(_col0)) (type: double), _col10 (type: tinyint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
                   Statistics: Num rows: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorization_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorization_15.q.out b/ql/src/test/results/clientpositive/tez/vectorization_15.q.out
index 1858cb0..c134c0b 100644
--- a/ql/src/test/results/clientpositive/tez/vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorization_15.q.out
@@ -85,19 +85,19 @@ STAGE PLANS:
                     predicate: ((cstring2 like '%ss%') or (cstring1 like '10%') or ((cint >= -75) and (UDFToShort(ctinyint) = csmallint) and (cdouble >= -3728.0))) (type: boolean)
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: cfloat (type: float), cboolean1 (type: boolean), cdouble (type: double), cstring1 (type: string), ctinyint (type: tinyint), cint (type: int), ctimestamp1 (type: timestamp)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                      expressions: ctinyint (type: tinyint), cint (type: int), cfloat (type: float), cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
+                      outputColumnNames: ctinyint, cint, cfloat, cdouble, cstring1, ctimestamp1, cboolean1
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: stddev_samp(_col0), min(_col2), stddev_samp(_col4), var_pop(_col4), var_samp(_col5), stddev_pop(_col5)
-                        keys: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
+                        aggregations: stddev_samp(cfloat), min(cdouble), stddev_samp(ctinyint), var_pop(ctinyint), var_samp(cint), stddev_pop(cint)
+                        keys: ctinyint (type: tinyint), cint (type: int), cfloat (type: float), cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                         Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
+                          key expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: float), _col3 (type: double), _col4 (type: string), _col5 (type: timestamp), _col6 (type: boolean)
                           sort order: +++++++
-                          Map-reduce partition columns: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
+                          Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: int), _col2 (type: float), _col3 (type: double), _col4 (type: string), _col5 (type: timestamp), _col6 (type: boolean)
                           Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: double), _col9 (type: struct<count:bigint,sum:double,variance:double>), _col10 (type: struct<count:bigint,sum:double,variance:double>), _col11 (type: struct<count:bigint,sum:double,variance:double>), _col12 (type: struct<count:bigint,sum:double,variance:double>)
             Execution mode: vectorized
@@ -105,12 +105,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: stddev_samp(VALUE._col0), min(VALUE._col1), stddev_samp(VALUE._col2), var_pop(VALUE._col3), var_samp(VALUE._col4), stddev_pop(VALUE._col5)
-                keys: KEY._col0 (type: float), KEY._col1 (type: boolean), KEY._col2 (type: double), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int), KEY._col6 (type: timestamp)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: int), KEY._col2 (type: float), KEY._col3 (type: double), KEY._col4 (type: string), KEY._col5 (type: timestamp), KEY._col6 (type: boolean)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp), _col7 (type: double), (-26.28 - UDFToDouble(_col5)) (type: double), _col8 (type: double), (_col2 * 79.553) (type: double), (33.0 % _col0) (type: float), _col9 (type: double), _col10 (type: double), (-23.0 % _col2) (type: double), (- _col4) (type: tinyint), _col11 (type: double), (UDFToFloat(_col5) - _col0) (type: float), (-23 % UDFToInteger(_col4)) (type: int), (- (-26.28 - UDFToDouble(_col5))) (type: double), _col12 (type: double)
+                  expressions: _col2 (type: float), _col6 (type: boolean), _col3 (type: double), _col4 (type: string), _col0 (type: tinyint), _col1 (type: int), _col5 (type: timestamp), _col7 (type: double), (-26.28 - UDFToDouble(_col1)) (type: double), _col8 (type: double), (_col3 * 79.553) (type: double), (33.0 % _col2) (type: float), _col9 (type: double), _col10 (type: double), (-23.0 % _col3) (type: double), (- _col0) (type: tinyint), _col11 (type: double), (UDFToFloat(_col1) - _col2) (type: float), (-23 % UDFToInteger(_col0)) (type: int), (- (-26.28 - UDFToDouble(_col1))) (type: double), _col12 (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
                   Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorization_16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorization_16.q.out b/ql/src/test/results/clientpositive/tez/vectorization_16.q.out
index bfed7d1..1b39796 100644
--- a/ql/src/test/results/clientpositive/tez/vectorization_16.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorization_16.q.out
@@ -62,19 +62,19 @@ STAGE PLANS:
                     predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389) or (cstring1 < 'a'))) (type: boolean)
                     Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp)
-                      outputColumnNames: _col0, _col1, _col2
+                      expressions: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
+                      outputColumnNames: cdouble, cstring1, ctimestamp1
                       Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: count(_col1), stddev_samp(_col1), min(_col1)
-                        keys: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                        aggregations: count(cdouble), stddev_samp(cdouble), min(cdouble)
+                        keys: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                         Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                          key expressions: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                           sort order: +++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                          Map-reduce partition columns: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                           Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col3 (type: bigint), _col4 (type: struct<count:bigint,sum:double,variance:double>), _col5 (type: double)
             Execution mode: vectorized
@@ -82,12 +82,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0), stddev_samp(VALUE._col1), min(VALUE._col2)
-                keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
+                keys: KEY._col0 (type: double), KEY._col1 (type: string), KEY._col2 (type: timestamp)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                 Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639) (type: double), (- (_col1 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col1) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
+                  expressions: _col1 (type: string), _col0 (type: double), _col2 (type: timestamp), (_col0 - 9763215.5639) (type: double), (- (_col0 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col0) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                   Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorization_9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorization_9.q.out b/ql/src/test/results/clientpositive/tez/vectorization_9.q.out
index bfed7d1..1b39796 100644
--- a/ql/src/test/results/clientpositive/tez/vectorization_9.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorization_9.q.out
@@ -62,19 +62,19 @@ STAGE PLANS:
                     predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389) or (cstring1 < 'a'))) (type: boolean)
                     Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp)
-                      outputColumnNames: _col0, _col1, _col2
+                      expressions: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
+                      outputColumnNames: cdouble, cstring1, ctimestamp1
                       Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: count(_col1), stddev_samp(_col1), min(_col1)
-                        keys: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                        aggregations: count(cdouble), stddev_samp(cdouble), min(cdouble)
+                        keys: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                         Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                          key expressions: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                           sort order: +++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                          Map-reduce partition columns: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                           Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col3 (type: bigint), _col4 (type: struct<count:bigint,sum:double,variance:double>), _col5 (type: double)
             Execution mode: vectorized
@@ -82,12 +82,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0), stddev_samp(VALUE._col1), min(VALUE._col2)
-                keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
+                keys: KEY._col0 (type: double), KEY._col1 (type: string), KEY._col2 (type: timestamp)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                 Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639) (type: double), (- (_col1 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col1) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
+                  expressions: _col1 (type: string), _col0 (type: double), _col2 (type: timestamp), (_col0 - 9763215.5639) (type: double), (- (_col0 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col0) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                   Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out b/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out
index 1c5b51f..33f7ed9 100644
--- a/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out
@@ -251,10 +251,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctinyint
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: tinyint)
+                      keys: ctinyint (type: tinyint)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -342,10 +342,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint), cdouble (type: double)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: ctinyint, cdouble
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: tinyint), _col1 (type: double)
+                      keys: ctinyint (type: tinyint), cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -469,11 +469,11 @@ STAGE PLANS:
                     Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cdouble (type: double), ctinyint (type: tinyint)
-                      outputColumnNames: _col0, _col1
+                      outputColumnNames: cdouble, ctinyint
                       Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: sum(_col1)
-                        keys: _col0 (type: double)
+                        aggregations: sum(ctinyint)
+                        keys: cdouble (type: double)
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorization_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorization_pushdown.q.out b/ql/src/test/results/clientpositive/tez/vectorization_pushdown.q.out
index e203bfd..c9c301e 100644
--- a/ql/src/test/results/clientpositive/tez/vectorization_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorization_pushdown.q.out
@@ -25,10 +25,10 @@ STAGE PLANS:
                     Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cbigint (type: bigint)
-                      outputColumnNames: _col0
+                      outputColumnNames: cbigint
                       Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: avg(_col0)
+                        aggregations: avg(cbigint)
                         mode: hash
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorization_short_regress.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorization_short_regress.q.out b/ql/src/test/results/clientpositive/tez/vectorization_short_regress.q.out
index 59b457a..99171f2 100644
--- a/ql/src/test/results/clientpositive/tez/vectorization_short_regress.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorization_short_regress.q.out
@@ -156,10 +156,10 @@ STAGE PLANS:
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cint (type: int), cdouble (type: double), csmallint (type: smallint), cfloat (type: float), ctinyint (type: tinyint)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      outputColumnNames: cint, cdouble, csmallint, cfloat, ctinyint
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: avg(_col0), sum(_col1), stddev_pop(_col0), stddev_samp(_col2), var_samp(_col0), avg(_col3), stddev_samp(_col0), min(_col4), count(_col2)
+                        aggregations: avg(cint), sum(cdouble), stddev_pop(cint), stddev_samp(csmallint), var_samp(cint), avg(cfloat), stddev_samp(cint), min(ctinyint), count(csmallint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                         Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
@@ -368,10 +368,10 @@ STAGE PLANS:
                     Statistics: Num rows: 6826 Data size: 1467614 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cint (type: int), cbigint (type: bigint), csmallint (type: smallint), cdouble (type: double), ctinyint (type: tinyint)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      outputColumnNames: cint, cbigint, csmallint, cdouble, ctinyint
                       Statistics: Num rows: 6826 Data size: 1467614 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: max(_col0), var_pop(_col1), stddev_pop(_col2), max(_col3), avg(_col4), min(_col0), min(_col3), stddev_samp(_col2), var_samp(_col0)
+                        aggregations: max(cint), var_pop(cbigint), stddev_pop(csmallint), max(cdouble), avg(ctinyint), min(cint), min(cdouble), stddev_samp(csmallint), var_samp(cint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -571,10 +571,10 @@ STAGE PLANS:
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cbigint (type: bigint), ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cdouble (type: double)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      outputColumnNames: cbigint, ctinyint, csmallint, cint, cdouble
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: var_pop(_col0), count(), max(_col1), stddev_pop(_col2), max(_col3), stddev_samp(_col4), count(_col1), avg(_col1)
+                        aggregations: var_pop(cbigint), count(), max(ctinyint), stddev_pop(csmallint), max(cint), stddev_samp(cdouble), count(ctinyint), avg(ctinyint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -753,10 +753,10 @@ STAGE PLANS:
                     Statistics: Num rows: 8874 Data size: 1907941 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: ctinyint (type: tinyint), cbigint (type: bigint), cint (type: int), cfloat (type: float)
-                      outputColumnNames: _col0, _col1, _col2, _col3
+                      outputColumnNames: ctinyint, cbigint, cint, cfloat
                       Statistics: Num rows: 8874 Data size: 1907941 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: avg(_col0), max(_col1), stddev_samp(_col2), var_pop(_col2), var_pop(_col1), max(_col3)
+                        aggregations: avg(ctinyint), max(cbigint), stddev_samp(cint), var_pop(cint), var_pop(cbigint), max(cfloat)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                         Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -1882,11 +1882,11 @@ STAGE PLANS:
                     Statistics: Num rows: 2503 Data size: 538153 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: csmallint (type: smallint), cbigint (type: bigint), ctinyint (type: tinyint)
-                      outputColumnNames: _col0, _col1, _col2
+                      outputColumnNames: csmallint, cbigint, ctinyint
                       Statistics: Num rows: 2503 Data size: 538153 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: stddev_samp(_col0), sum(_col1), var_pop(_col2), count()
-                        keys: _col0 (type: smallint)
+                        aggregations: stddev_samp(csmallint), sum(cbigint), var_pop(ctinyint), count()
+                        keys: csmallint (type: smallint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4
                         Statistics: Num rows: 2503 Data size: 538153 Basic stats: COMPLETE Column stats: NONE
@@ -2089,11 +2089,11 @@ STAGE PLANS:
                     Statistics: Num rows: 2654 Data size: 570619 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cdouble (type: double), cfloat (type: float)
-                      outputColumnNames: _col0, _col1
+                      outputColumnNames: cdouble, cfloat
                       Statistics: Num rows: 2654 Data size: 570619 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: var_samp(_col0), count(_col1), sum(_col1), var_pop(_col0), stddev_pop(_col0), sum(_col0)
-                        keys: _col0 (type: double)
+                        aggregations: var_samp(cdouble), count(cfloat), sum(cfloat), var_pop(cdouble), stddev_pop(cdouble), sum(cdouble)
+                        keys: cdouble (type: double)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                         Statistics: Num rows: 2654 Data size: 570619 Basic stats: COMPLETE Column stats: NONE
@@ -2343,19 +2343,19 @@ STAGE PLANS:
                     predicate: ((UDFToDouble(ctimestamp1) <> 0.0) and (((-257 <> UDFToInteger(ctinyint)) and cboolean2 is not null and cstring1 regexp '.*ss' and (-3.0 < UDFToDouble(ctimestamp1))) or (UDFToDouble(ctimestamp2) = -5.0) or ((UDFToDouble(ctimestamp1) < 0.0) and (cstring2 like '%b%')) or (cdouble = UDFToDouble(cint)) or (cboolean1 is null and (cfloat < UDFToFloat(cint))))) (type: boolean)
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: ctimestamp1 (type: timestamp), cstring1 (type: string), cint (type: int), csmallint (type: smallint), ctinyint (type: tinyint), cfloat (type: float), cdouble (type: double)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                      expressions: cstring1 (type: string), ctimestamp1 (type: timestamp), cint (type: int), csmallint (type: smallint), ctinyint (type: tinyint), cfloat (type: float), cdouble (type: double)
+                      outputColumnNames: cstring1, ctimestamp1, cint, csmallint, ctinyint, cfloat, cdouble
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: stddev_pop(_col2), avg(_col3), count(), min(_col4), var_samp(_col3), var_pop(_col5), avg(_col2), var_samp(_col5), avg(_col5), min(_col6), var_pop(_col3), stddev_pop(_col4), sum(_col2)
-                        keys: _col0 (type: timestamp), _col1 (type: string)
+                        aggregations: stddev_pop(cint), avg(csmallint), count(), min(ctinyint), var_samp(csmallint), var_pop(cfloat), avg(cint), var_samp(cfloat), avg(cfloat), min(cdouble), var_pop(csmallint), stddev_pop(ctinyint), sum(cint)
+                        keys: cstring1 (type: string), ctimestamp1 (type: timestamp)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
                         Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: timestamp), _col1 (type: string)
+                          key expressions: _col0 (type: string), _col1 (type: timestamp)
                           sort order: ++
-                          Map-reduce partition columns: _col0 (type: timestamp), _col1 (type: string)
+                          Map-reduce partition columns: _col0 (type: string), _col1 (type: timestamp)
                           Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col2 (type: struct<count:bigint,sum:double,variance:double>), _col3 (type: struct<count:bigint,sum:double,input:smallint>), _col4 (type: bigint), _col5 (type: tinyint), _col6 (type: struct<count:bigint,sum:double,variance:double>), _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: struct<count:bigint,sum:double,input:int>), _col9 (type: struct<count:bigint,sum:double,variance:double>), _col10 (type: struct<count:bigint,sum:double,input:float>), _col11 (type: double), _col12 (type: struct<count:bigint,sum:double,variance:double>), _col13 (type: struct<count:bigint,sum:double,variance:double>), _col14 (type: bigint)
             Execution mode: vectorized
@@ -2363,12 +2363,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: stddev_pop(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), min(VALUE._col3), var_samp(VALUE._col4), var_pop(VALUE._col5), avg(VALUE._col6), var_samp(VALUE._col7), avg(VALUE._col8), min(VALUE._col9), var_pop(VALUE._col10), stddev_pop(VALUE._col11), sum(VALUE._col12)
-                keys: KEY._col0 (type: timestamp), KEY._col1 (type: string)
+                keys: KEY._col0 (type: string), KEY._col1 (type: timestamp)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
                 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: timestamp), _col1 (type: string), _col2 (type: double), (_col2 * 10.175) (type: double), (- _col2) (type: double), _col3 (type: double), (- _col2) (type: double), (-26.28 - _col2) (type: double), _col4 (type: bigint), (- _col4) (type: bigint), ((-26.28 - _col2) * (- _col2)) (type: double), _col5 (type: tinyint), (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4))) (type: double), (- (_col2 * 10.175)) (type: double), _col6 (type: double), (_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- _col2)) (type: double), (UDFToDouble((- _col4)) / _col2) (type: double), _col7 (type: double), (10.175 / _col3) (type: double), _col8 (type: double), _col9 (type: double), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- (_col2 * 10.175))) (type: double), _col10 (type: double), (((_col6 + (((-26.28 - _col2) * (-
  _col2)) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) * 10.175) (type: double), (10.175 % (10.175 / _col3)) (type: double), (- _col5) (type: tinyint), _col11 (type: double), _col12 (type: double), (- ((-26.28 - _col2) * (- _col2))) (type: double), ((- _col2) % _col10) (type: double), (-26.28 / UDFToDouble((- _col5))) (type: double), _col13 (type: double), _col14 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) / _col7) (type: double), (- (- _col4)) (type: bigint), _col4 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) % -26.28) (type: double)
+                  expressions: _col1 (type: timestamp), _col0 (type: string), _col2 (type: double), (_col2 * 10.175) (type: double), (- _col2) (type: double), _col3 (type: double), (- _col2) (type: double), (-26.28 - _col2) (type: double), _col4 (type: bigint), (- _col4) (type: bigint), ((-26.28 - _col2) * (- _col2)) (type: double), _col5 (type: tinyint), (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4))) (type: double), (- (_col2 * 10.175)) (type: double), _col6 (type: double), (_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- _col2)) (type: double), (UDFToDouble((- _col4)) / _col2) (type: double), _col7 (type: double), (10.175 / _col3) (type: double), _col8 (type: double), _col9 (type: double), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- (_col2 * 10.175))) (type: double), _col10 (type: double), (((_col6 + (((-26.28 - _col2) * (-
  _col2)) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) * 10.175) (type: double), (10.175 % (10.175 / _col3)) (type: double), (- _col5) (type: tinyint), _col11 (type: double), _col12 (type: double), (- ((-26.28 - _col2) * (- _col2))) (type: double), ((- _col2) % _col10) (type: double), (-26.28 / UDFToDouble((- _col5))) (type: double), _col13 (type: double), _col14 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) / _col7) (type: double), (- (- _col4)) (type: bigint), _col4 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) % -26.28) (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38
                   Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
@@ -2676,11 +2676,11 @@ STAGE PLANS:
                     Statistics: Num rows: 4778 Data size: 1027287 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cboolean1 (type: boolean), cfloat (type: float), cbigint (type: bigint), cint (type: int), cdouble (type: double), ctinyint (type: tinyint), csmallint (type: smallint)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                      outputColumnNames: cboolean1, cfloat, cbigint, cint, cdouble, ctinyint, csmallint
                       Statistics: Num rows: 4778 Data size: 1027287 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: max(_col1), sum(_col2), var_samp(_col3), avg(_col4), min(_col2), var_pop(_col2), sum(_col3), stddev_samp(_col5), stddev_pop(_col6), avg(_col3)
-                        keys: _col0 (type: boolean)
+                        aggregations: max(cfloat), sum(cbigint), var_samp(cint), avg(cdouble), min(cbigint), var_pop(cbigint), sum(cint), stddev_samp(ctinyint), stddev_pop(csmallint), avg(cint)
+                        keys: cboolean1 (type: boolean)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                         Statistics: Num rows: 4778 Data size: 1027287 Basic stats: COMPLETE Column stats: NONE
@@ -2915,10 +2915,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
                     expressions: i (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: i
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(i)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3093,10 +3093,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctinyint
                     Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(ctinyint)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3160,10 +3160,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cint (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: cint
                     Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(cint)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3227,10 +3227,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cfloat (type: float)
-                    outputColumnNames: _col0
+                    outputColumnNames: cfloat
                     Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(cfloat)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3294,10 +3294,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cstring1 (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: cstring1
                     Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(cstring1)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3361,10 +3361,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cboolean1 (type: boolean)
-                    outputColumnNames: _col0
+                    outputColumnNames: cboolean1
                     Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(cboolean1)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorized_distinct_gby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorized_distinct_gby.q.out b/ql/src/test/results/clientpositive/tez/vectorized_distinct_gby.q.out
index 23798e0..468802c 100644
--- a/ql/src/test/results/clientpositive/tez/vectorized_distinct_gby.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorized_distinct_gby.q.out
@@ -38,10 +38,10 @@ STAGE PLANS:
                   Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: a (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: a
                     Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int)
+                      keys: a (type: int)
                       mode: final
                       outputColumnNames: _col0
                       Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE
@@ -109,10 +109,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cint (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: cint
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int)
+                      keys: cint (type: int)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out
index b253508..f0ddc5b 100644
--- a/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out
@@ -59,10 +59,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string)
+                      keys: ds (type: string)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
@@ -1793,7 +1793,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 #### A masked pattern was here ####
 1000
-Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: -- parent is reduce tasks
 EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'
 PREHOOK: type: QUERY
@@ -1832,10 +1832,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: '2008-04-08' (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string)
+                      keys: ds (type: string)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -1897,7 +1897,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@srcpart
@@ -2773,10 +2773,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -2791,10 +2791,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0)
+                      aggregations: min(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -2973,10 +2973,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -2991,10 +2991,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0)
+                      aggregations: min(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -3161,20 +3161,16 @@ STAGE PLANS:
                   alias: srcpart
                   filterExpr: ds is not null (type: boolean)
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: ds (type: string)
+                  Group By Operator
+                    keys: ds (type: string)
+                    mode: hash
                     outputColumnNames: _col0
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                    Group By Operator
-                      keys: _col0 (type: string)
-                      mode: hash
-                      outputColumnNames: _col0
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
         Map 10 
             Map Operator Tree:
                 TableScan
@@ -3182,10 +3178,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0)
+                      aggregations: min(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -3199,20 +3195,16 @@ STAGE PLANS:
                   alias: srcpart
                   filterExpr: ds is not null (type: boolean)
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: ds (type: string)
+                  Group By Operator
+                    keys: ds (type: string)
+                    mode: hash
                     outputColumnNames: _col0
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                    Group By Operator
-                      keys: _col0 (type: string)
-                      mode: hash
-                      outputColumnNames: _col0
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
         Map 7 
             Map Operator Tree:
                 TableScan
@@ -3220,10 +3212,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -4186,7 +4178,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 #### A masked pattern was here ####
 1000
-Warning: Map Join MAPJOIN[23][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Map 1' is a cross product
 PREHOOK: query: -- parent is reduce tasks
 EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'
 PREHOOK: type: QUERY
@@ -4241,10 +4233,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: '2008-04-08' (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string)
+                      keys: ds (type: string)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -4288,7 +4280,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Map Join MAPJOIN[23][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Map 1' is a cross product
 PREHOOK: query: select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@srcpart
@@ -4889,10 +4881,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col0)
+                      aggregations: max(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -4907,10 +4899,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ds (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: ds
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0)
+                      aggregations: min(ds)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/tez/vectorized_nested_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorized_nested_mapjoin.q.out b/ql/src/test/results/clientpositive/tez/vectorized_nested_mapjoin.q.out
index 01a36e5..74be17b 100644
--- a/ql/src/test/results/clientpositive/tez/vectorized_nested_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorized_nested_mapjoin.q.out
@@ -75,19 +75,15 @@ STAGE PLANS:
                               1 Map 4
                             Statistics: Num rows: 7433 Data size: 1598388 Basic stats: COMPLETE Column stats: NONE
                             HybridGraceHashJoin: true
-                            Select Operator
-                              expressions: _col1 (type: double)
+                            Group By Operator
+                              aggregations: sum(_col1)
+                              mode: hash
                               outputColumnNames: _col0
-                              Statistics: Num rows: 7433 Data size: 1598388 Basic stats: COMPLETE Column stats: NONE
-                              Group By Operator
-                                aggregations: sum(_col0)
-                                mode: hash
-                                outputColumnNames: _col0
+                              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                              Reduce Output Operator
+                                sort order: 
                                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                                Reduce Output Operator
-                                  sort order: 
-                                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                                  value expressions: _col0 (type: double)
+                                value expressions: _col0 (type: double)
             Execution mode: vectorized
         Map 4 
             Map Operator Tree:


[43/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vectorization_13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorization_13.q.out b/ql/src/test/results/clientpositive/spark/vectorization_13.q.out
index 22be1d7..122e87a 100644
--- a/ql/src/test/results/clientpositive/spark/vectorization_13.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorization_13.q.out
@@ -89,19 +89,19 @@ STAGE PLANS:
                     predicate: (((cfloat < 3569.0) and (10.175 >= cdouble) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > 11.0) and (UDFToDouble(ctimestamp2) <> 12.0) and (UDFToDouble(ctinyint) < 9763215.5639))) (type: boolean)
                     Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      expressions: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
+                      outputColumnNames: ctinyint, cfloat, cstring1, ctimestamp1, cboolean1
                       Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: max(_col1), sum(_col3), stddev_pop(_col3), stddev_pop(_col1), max(_col3), min(_col1)
-                        keys: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                        aggregations: max(ctinyint), sum(cfloat), stddev_pop(cfloat), stddev_pop(ctinyint), max(cfloat), min(ctinyint)
+                        keys: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                         Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                          key expressions: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                           sort order: +++++
-                          Map-reduce partition columns: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                          Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                           Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: struct<count:bigint,sum:double,variance:double>), _col9 (type: float), _col10 (type: tinyint)
             Execution mode: vectorized
@@ -109,12 +109,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: max(VALUE._col0), sum(VALUE._col1), stddev_pop(VALUE._col2), stddev_pop(VALUE._col3), max(VALUE._col4), min(VALUE._col5)
-                keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: float), KEY._col2 (type: string), KEY._col3 (type: timestamp), KEY._col4 (type: boolean)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                 Statistics: Num rows: 1365 Data size: 41904 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col3)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col1) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col1) + _col5))) / UDFToDouble(_col1)) (type: double), _col10 (type: tinyint)
+                  expressions: _col4 (type: boolean), _col0 (type: tinyint), _col3 (type: timestamp), _col1 (type: float), _col2 (type: string), (- _col0) (type: tinyint), _col5 (type: tinyint), ((- _col0) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col0) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col1)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col0) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col0) + _col5))) / UDFToDouble(_col0)) (type: double), _col10 (type: tinyint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
                   Statistics: Num rows: 1365 Data size: 41904 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
@@ -342,19 +342,19 @@ STAGE PLANS:
                     predicate: (((cfloat < 3569.0) and (10.175 >= cdouble) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -1.388) and (UDFToDouble(ctimestamp2) <> -1.3359999999999999) and (UDFToDouble(ctinyint) < 9763215.5639))) (type: boolean)
                     Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      expressions: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
+                      outputColumnNames: ctinyint, cfloat, cstring1, ctimestamp1, cboolean1
                       Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: max(_col1), sum(_col3), stddev_pop(_col3), stddev_pop(_col1), max(_col3), min(_col1)
-                        keys: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                        aggregations: max(ctinyint), sum(cfloat), stddev_pop(cfloat), stddev_pop(ctinyint), max(cfloat), min(ctinyint)
+                        keys: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                         Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                          key expressions: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                           sort order: +++++
-                          Map-reduce partition columns: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                          Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                           Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: struct<count:bigint,sum:double,variance:double>), _col9 (type: float), _col10 (type: tinyint)
             Execution mode: vectorized
@@ -362,12 +362,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: max(VALUE._col0), sum(VALUE._col1), stddev_pop(VALUE._col2), stddev_pop(VALUE._col3), max(VALUE._col4), min(VALUE._col5)
-                keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: float), KEY._col2 (type: string), KEY._col3 (type: timestamp), KEY._col4 (type: boolean)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                 Statistics: Num rows: 1365 Data size: 41904 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col3)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col1) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col1) + _col5))) / UDFToDouble(_col1)) (type: double), _col10 (type: tinyint)
+                  expressions: _col4 (type: boolean), _col0 (type: tinyint), _col3 (type: timestamp), _col1 (type: float), _col2 (type: string), (- _col0) (type: tinyint), _col5 (type: tinyint), ((- _col0) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col0) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col1)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col0) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col0) + _col5))) / UDFToDouble(_col0)) (type: double), _col10 (type: tinyint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
                   Statistics: Num rows: 1365 Data size: 41904 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vectorization_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorization_15.q.out b/ql/src/test/results/clientpositive/spark/vectorization_15.q.out
index 8013bfe..ec77280 100644
--- a/ql/src/test/results/clientpositive/spark/vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorization_15.q.out
@@ -85,19 +85,19 @@ STAGE PLANS:
                     predicate: ((cstring2 like '%ss%') or (cstring1 like '10%') or ((cint >= -75) and (UDFToShort(ctinyint) = csmallint) and (cdouble >= -3728.0))) (type: boolean)
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: cfloat (type: float), cboolean1 (type: boolean), cdouble (type: double), cstring1 (type: string), ctinyint (type: tinyint), cint (type: int), ctimestamp1 (type: timestamp)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                      expressions: ctinyint (type: tinyint), cint (type: int), cfloat (type: float), cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
+                      outputColumnNames: ctinyint, cint, cfloat, cdouble, cstring1, ctimestamp1, cboolean1
                       Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: stddev_samp(_col0), min(_col2), stddev_samp(_col4), var_pop(_col4), var_samp(_col5), stddev_pop(_col5)
-                        keys: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
+                        aggregations: stddev_samp(cfloat), min(cdouble), stddev_samp(ctinyint), var_pop(ctinyint), var_samp(cint), stddev_pop(cint)
+                        keys: ctinyint (type: tinyint), cint (type: int), cfloat (type: float), cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                         Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
+                          key expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: float), _col3 (type: double), _col4 (type: string), _col5 (type: timestamp), _col6 (type: boolean)
                           sort order: +++++++
-                          Map-reduce partition columns: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
+                          Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: int), _col2 (type: float), _col3 (type: double), _col4 (type: string), _col5 (type: timestamp), _col6 (type: boolean)
                           Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: double), _col9 (type: struct<count:bigint,sum:double,variance:double>), _col10 (type: struct<count:bigint,sum:double,variance:double>), _col11 (type: struct<count:bigint,sum:double,variance:double>), _col12 (type: struct<count:bigint,sum:double,variance:double>)
             Execution mode: vectorized
@@ -105,12 +105,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: stddev_samp(VALUE._col0), min(VALUE._col1), stddev_samp(VALUE._col2), var_pop(VALUE._col3), var_samp(VALUE._col4), stddev_pop(VALUE._col5)
-                keys: KEY._col0 (type: float), KEY._col1 (type: boolean), KEY._col2 (type: double), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int), KEY._col6 (type: timestamp)
+                keys: KEY._col0 (type: tinyint), KEY._col1 (type: int), KEY._col2 (type: float), KEY._col3 (type: double), KEY._col4 (type: string), KEY._col5 (type: timestamp), KEY._col6 (type: boolean)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 6144 Data size: 188618 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp), _col7 (type: double), (-26.28 - UDFToDouble(_col5)) (type: double), _col8 (type: double), (_col2 * 79.553) (type: double), (33.0 % _col0) (type: float), _col9 (type: double), _col10 (type: double), (-23.0 % _col2) (type: double), (- _col4) (type: tinyint), _col11 (type: double), (UDFToFloat(_col5) - _col0) (type: float), (-23 % UDFToInteger(_col4)) (type: int), (- (-26.28 - UDFToDouble(_col5))) (type: double), _col12 (type: double)
+                  expressions: _col2 (type: float), _col6 (type: boolean), _col3 (type: double), _col4 (type: string), _col0 (type: tinyint), _col1 (type: int), _col5 (type: timestamp), _col7 (type: double), (-26.28 - UDFToDouble(_col1)) (type: double), _col8 (type: double), (_col3 * 79.553) (type: double), (33.0 % _col2) (type: float), _col9 (type: double), _col10 (type: double), (-23.0 % _col3) (type: double), (- _col0) (type: tinyint), _col11 (type: double), (UDFToFloat(_col1) - _col2) (type: float), (-23 % UDFToInteger(_col0)) (type: int), (- (-26.28 - UDFToDouble(_col1))) (type: double), _col12 (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
                   Statistics: Num rows: 6144 Data size: 188618 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vectorization_16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorization_16.q.out b/ql/src/test/results/clientpositive/spark/vectorization_16.q.out
index a42c30a..3326044 100644
--- a/ql/src/test/results/clientpositive/spark/vectorization_16.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorization_16.q.out
@@ -62,19 +62,19 @@ STAGE PLANS:
                     predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389) or (cstring1 < 'a'))) (type: boolean)
                     Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp)
-                      outputColumnNames: _col0, _col1, _col2
+                      expressions: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
+                      outputColumnNames: cdouble, cstring1, ctimestamp1
                       Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: count(_col1), stddev_samp(_col1), min(_col1)
-                        keys: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                        aggregations: count(cdouble), stddev_samp(cdouble), min(cdouble)
+                        keys: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                         Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                          key expressions: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                           sort order: +++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                          Map-reduce partition columns: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                           Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col3 (type: bigint), _col4 (type: struct<count:bigint,sum:double,variance:double>), _col5 (type: double)
             Execution mode: vectorized
@@ -82,12 +82,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0), stddev_samp(VALUE._col1), min(VALUE._col2)
-                keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
+                keys: KEY._col0 (type: double), KEY._col1 (type: string), KEY._col2 (type: timestamp)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                 Statistics: Num rows: 2048 Data size: 62872 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639) (type: double), (- (_col1 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col1) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
+                  expressions: _col1 (type: string), _col0 (type: double), _col2 (type: timestamp), (_col0 - 9763215.5639) (type: double), (- (_col0 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col0) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                   Statistics: Num rows: 2048 Data size: 62872 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vectorization_9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorization_9.q.out b/ql/src/test/results/clientpositive/spark/vectorization_9.q.out
index a42c30a..3326044 100644
--- a/ql/src/test/results/clientpositive/spark/vectorization_9.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorization_9.q.out
@@ -62,19 +62,19 @@ STAGE PLANS:
                     predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389) or (cstring1 < 'a'))) (type: boolean)
                     Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp)
-                      outputColumnNames: _col0, _col1, _col2
+                      expressions: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
+                      outputColumnNames: cdouble, cstring1, ctimestamp1
                       Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: count(_col1), stddev_samp(_col1), min(_col1)
-                        keys: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                        aggregations: count(cdouble), stddev_samp(cdouble), min(cdouble)
+                        keys: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                         Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                          key expressions: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                           sort order: +++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                          Map-reduce partition columns: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                           Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col3 (type: bigint), _col4 (type: struct<count:bigint,sum:double,variance:double>), _col5 (type: double)
             Execution mode: vectorized
@@ -82,12 +82,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0), stddev_samp(VALUE._col1), min(VALUE._col2)
-                keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
+                keys: KEY._col0 (type: double), KEY._col1 (type: string), KEY._col2 (type: timestamp)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                 Statistics: Num rows: 2048 Data size: 62872 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639) (type: double), (- (_col1 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col1) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
+                  expressions: _col1 (type: string), _col0 (type: double), _col2 (type: timestamp), (_col0 - 9763215.5639) (type: double), (- (_col0 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col0) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                   Statistics: Num rows: 2048 Data size: 62872 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vectorization_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorization_pushdown.q.out b/ql/src/test/results/clientpositive/spark/vectorization_pushdown.q.out
index eb8914b..59caac7 100644
--- a/ql/src/test/results/clientpositive/spark/vectorization_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorization_pushdown.q.out
@@ -25,10 +25,10 @@ STAGE PLANS:
                     Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cbigint (type: bigint)
-                      outputColumnNames: _col0
+                      outputColumnNames: cbigint
                       Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: avg(_col0)
+                        aggregations: avg(cbigint)
                         mode: hash
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out b/ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out
index 3d17aba..7b4f846 100644
--- a/ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out
@@ -156,10 +156,10 @@ STAGE PLANS:
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cint (type: int), cdouble (type: double), csmallint (type: smallint), cfloat (type: float), ctinyint (type: tinyint)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      outputColumnNames: cint, cdouble, csmallint, cfloat, ctinyint
                       Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: avg(_col0), sum(_col1), stddev_pop(_col0), stddev_samp(_col2), var_samp(_col0), avg(_col3), stddev_samp(_col0), min(_col4), count(_col2)
+                        aggregations: avg(cint), sum(cdouble), stddev_pop(cint), stddev_samp(csmallint), var_samp(cint), avg(cfloat), stddev_samp(cint), min(ctinyint), count(csmallint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                         Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
@@ -368,10 +368,10 @@ STAGE PLANS:
                     Statistics: Num rows: 6826 Data size: 209555 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cint (type: int), cbigint (type: bigint), csmallint (type: smallint), cdouble (type: double), ctinyint (type: tinyint)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      outputColumnNames: cint, cbigint, csmallint, cdouble, ctinyint
                       Statistics: Num rows: 6826 Data size: 209555 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: max(_col0), var_pop(_col1), stddev_pop(_col2), max(_col3), avg(_col4), min(_col0), min(_col3), stddev_samp(_col2), var_samp(_col0)
+                        aggregations: max(cint), var_pop(cbigint), stddev_pop(csmallint), max(cdouble), avg(ctinyint), min(cint), min(cdouble), stddev_samp(csmallint), var_samp(cint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -571,10 +571,10 @@ STAGE PLANS:
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cbigint (type: bigint), ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cdouble (type: double)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      outputColumnNames: cbigint, ctinyint, csmallint, cint, cdouble
                       Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: var_pop(_col0), count(), max(_col1), stddev_pop(_col2), max(_col3), stddev_samp(_col4), count(_col1), avg(_col1)
+                        aggregations: var_pop(cbigint), count(), max(ctinyint), stddev_pop(csmallint), max(cint), stddev_samp(cdouble), count(ctinyint), avg(ctinyint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -753,10 +753,10 @@ STAGE PLANS:
                     Statistics: Num rows: 8874 Data size: 272428 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: ctinyint (type: tinyint), cbigint (type: bigint), cint (type: int), cfloat (type: float)
-                      outputColumnNames: _col0, _col1, _col2, _col3
+                      outputColumnNames: ctinyint, cbigint, cint, cfloat
                       Statistics: Num rows: 8874 Data size: 272428 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: avg(_col0), max(_col1), stddev_samp(_col2), var_pop(_col2), var_pop(_col1), max(_col3)
+                        aggregations: avg(ctinyint), max(cbigint), stddev_samp(cint), var_pop(cint), var_pop(cbigint), max(cfloat)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                         Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -1882,11 +1882,11 @@ STAGE PLANS:
                     Statistics: Num rows: 2503 Data size: 76841 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: csmallint (type: smallint), cbigint (type: bigint), ctinyint (type: tinyint)
-                      outputColumnNames: _col0, _col1, _col2
+                      outputColumnNames: csmallint, cbigint, ctinyint
                       Statistics: Num rows: 2503 Data size: 76841 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: stddev_samp(_col0), sum(_col1), var_pop(_col2), count()
-                        keys: _col0 (type: smallint)
+                        aggregations: stddev_samp(csmallint), sum(cbigint), var_pop(ctinyint), count()
+                        keys: csmallint (type: smallint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4
                         Statistics: Num rows: 2503 Data size: 76841 Basic stats: COMPLETE Column stats: NONE
@@ -2089,11 +2089,11 @@ STAGE PLANS:
                     Statistics: Num rows: 2654 Data size: 81476 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cdouble (type: double), cfloat (type: float)
-                      outputColumnNames: _col0, _col1
+                      outputColumnNames: cdouble, cfloat
                       Statistics: Num rows: 2654 Data size: 81476 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: var_samp(_col0), count(_col1), sum(_col1), var_pop(_col0), stddev_pop(_col0), sum(_col0)
-                        keys: _col0 (type: double)
+                        aggregations: var_samp(cdouble), count(cfloat), sum(cfloat), var_pop(cdouble), stddev_pop(cdouble), sum(cdouble)
+                        keys: cdouble (type: double)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                         Statistics: Num rows: 2654 Data size: 81476 Basic stats: COMPLETE Column stats: NONE
@@ -2343,19 +2343,19 @@ STAGE PLANS:
                     predicate: ((UDFToDouble(ctimestamp1) <> 0.0) and (((-257 <> UDFToInteger(ctinyint)) and cboolean2 is not null and cstring1 regexp '.*ss' and (-3.0 < UDFToDouble(ctimestamp1))) or (UDFToDouble(ctimestamp2) = -5.0) or ((UDFToDouble(ctimestamp1) < 0.0) and (cstring2 like '%b%')) or (cdouble = UDFToDouble(cint)) or (cboolean1 is null and (cfloat < UDFToFloat(cint))))) (type: boolean)
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: ctimestamp1 (type: timestamp), cstring1 (type: string), cint (type: int), csmallint (type: smallint), ctinyint (type: tinyint), cfloat (type: float), cdouble (type: double)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                      expressions: cstring1 (type: string), ctimestamp1 (type: timestamp), cint (type: int), csmallint (type: smallint), ctinyint (type: tinyint), cfloat (type: float), cdouble (type: double)
+                      outputColumnNames: cstring1, ctimestamp1, cint, csmallint, ctinyint, cfloat, cdouble
                       Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: stddev_pop(_col2), avg(_col3), count(), min(_col4), var_samp(_col3), var_pop(_col5), avg(_col2), var_samp(_col5), avg(_col5), min(_col6), var_pop(_col3), stddev_pop(_col4), sum(_col2)
-                        keys: _col0 (type: timestamp), _col1 (type: string)
+                        aggregations: stddev_pop(cint), avg(csmallint), count(), min(ctinyint), var_samp(csmallint), var_pop(cfloat), avg(cint), var_samp(cfloat), avg(cfloat), min(cdouble), var_pop(csmallint), stddev_pop(ctinyint), sum(cint)
+                        keys: cstring1 (type: string), ctimestamp1 (type: timestamp)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
                         Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: timestamp), _col1 (type: string)
+                          key expressions: _col0 (type: string), _col1 (type: timestamp)
                           sort order: ++
-                          Map-reduce partition columns: _col0 (type: timestamp), _col1 (type: string)
+                          Map-reduce partition columns: _col0 (type: string), _col1 (type: timestamp)
                           Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col2 (type: struct<count:bigint,sum:double,variance:double>), _col3 (type: struct<count:bigint,sum:double,input:smallint>), _col4 (type: bigint), _col5 (type: tinyint), _col6 (type: struct<count:bigint,sum:double,variance:double>), _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: struct<count:bigint,sum:double,input:int>), _col9 (type: struct<count:bigint,sum:double,variance:double>), _col10 (type: struct<count:bigint,sum:double,input:float>), _col11 (type: double), _col12 (type: struct<count:bigint,sum:double,variance:double>), _col13 (type: struct<count:bigint,sum:double,variance:double>), _col14 (type: bigint)
             Execution mode: vectorized
@@ -2363,12 +2363,12 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: stddev_pop(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), min(VALUE._col3), var_samp(VALUE._col4), var_pop(VALUE._col5), avg(VALUE._col6), var_samp(VALUE._col7), avg(VALUE._col8), min(VALUE._col9), var_pop(VALUE._col10), stddev_pop(VALUE._col11), sum(VALUE._col12)
-                keys: KEY._col0 (type: timestamp), KEY._col1 (type: string)
+                keys: KEY._col0 (type: string), KEY._col1 (type: timestamp)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
                 Statistics: Num rows: 6144 Data size: 188618 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: timestamp), _col1 (type: string), _col2 (type: double), (_col2 * 10.175) (type: double), (- _col2) (type: double), _col3 (type: double), (- _col2) (type: double), (-26.28 - _col2) (type: double), _col4 (type: bigint), (- _col4) (type: bigint), ((-26.28 - _col2) * (- _col2)) (type: double), _col5 (type: tinyint), (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4))) (type: double), (- (_col2 * 10.175)) (type: double), _col6 (type: double), (_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- _col2)) (type: double), (UDFToDouble((- _col4)) / _col2) (type: double), _col7 (type: double), (10.175 / _col3) (type: double), _col8 (type: double), _col9 (type: double), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- (_col2 * 10.175))) (type: double), _col10 (type: double), (((_col6 + (((-26.28 - _col2) * (-
  _col2)) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) * 10.175) (type: double), (10.175 % (10.175 / _col3)) (type: double), (- _col5) (type: tinyint), _col11 (type: double), _col12 (type: double), (- ((-26.28 - _col2) * (- _col2))) (type: double), ((- _col2) % _col10) (type: double), (-26.28 / UDFToDouble((- _col5))) (type: double), _col13 (type: double), _col14 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) / _col7) (type: double), (- (- _col4)) (type: bigint), _col4 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) % -26.28) (type: double)
+                  expressions: _col1 (type: timestamp), _col0 (type: string), _col2 (type: double), (_col2 * 10.175) (type: double), (- _col2) (type: double), _col3 (type: double), (- _col2) (type: double), (-26.28 - _col2) (type: double), _col4 (type: bigint), (- _col4) (type: bigint), ((-26.28 - _col2) * (- _col2)) (type: double), _col5 (type: tinyint), (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4))) (type: double), (- (_col2 * 10.175)) (type: double), _col6 (type: double), (_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- _col2)) (type: double), (UDFToDouble((- _col4)) / _col2) (type: double), _col7 (type: double), (10.175 / _col3) (type: double), _col8 (type: double), _col9 (type: double), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- (_col2 * 10.175))) (type: double), _col10 (type: double), (((_col6 + (((-26.28 - _col2) * (-
  _col2)) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) * 10.175) (type: double), (10.175 % (10.175 / _col3)) (type: double), (- _col5) (type: tinyint), _col11 (type: double), _col12 (type: double), (- ((-26.28 - _col2) * (- _col2))) (type: double), ((- _col2) % _col10) (type: double), (-26.28 / UDFToDouble((- _col5))) (type: double), _col13 (type: double), _col14 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) / _col7) (type: double), (- (- _col4)) (type: bigint), _col4 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) % -26.28) (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38
                   Statistics: Num rows: 6144 Data size: 188618 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
@@ -2676,11 +2676,11 @@ STAGE PLANS:
                     Statistics: Num rows: 4778 Data size: 146682 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cboolean1 (type: boolean), cfloat (type: float), cbigint (type: bigint), cint (type: int), cdouble (type: double), ctinyint (type: tinyint), csmallint (type: smallint)
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                      outputColumnNames: cboolean1, cfloat, cbigint, cint, cdouble, ctinyint, csmallint
                       Statistics: Num rows: 4778 Data size: 146682 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: max(_col1), sum(_col2), var_samp(_col3), avg(_col4), min(_col2), var_pop(_col2), sum(_col3), stddev_samp(_col5), stddev_pop(_col6), avg(_col3)
-                        keys: _col0 (type: boolean)
+                        aggregations: max(cfloat), sum(cbigint), var_samp(cint), avg(cdouble), min(cbigint), var_pop(cbigint), sum(cint), stddev_samp(ctinyint), stddev_pop(csmallint), avg(cint)
+                        keys: cboolean1 (type: boolean)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                         Statistics: Num rows: 4778 Data size: 146682 Basic stats: COMPLETE Column stats: NONE
@@ -2915,10 +2915,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
                     expressions: i (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: i
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(i)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3093,10 +3093,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctinyint
                     Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(ctinyint)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3160,10 +3160,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cint (type: int)
-                    outputColumnNames: _col0
+                    outputColumnNames: cint
                     Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(cint)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3227,10 +3227,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cfloat (type: float)
-                    outputColumnNames: _col0
+                    outputColumnNames: cfloat
                     Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(cfloat)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3294,10 +3294,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cstring1 (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: cstring1
                     Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(cstring1)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3361,10 +3361,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cboolean1 (type: boolean)
-                    outputColumnNames: _col0
+                    outputColumnNames: cboolean1
                     Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(cboolean1)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vectorized_nested_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorized_nested_mapjoin.q.out b/ql/src/test/results/clientpositive/spark/vectorized_nested_mapjoin.q.out
index 7ba64b7..316ed63 100644
--- a/ql/src/test/results/clientpositive/spark/vectorized_nested_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorized_nested_mapjoin.q.out
@@ -97,19 +97,15 @@ STAGE PLANS:
                             input vertices:
                               1 Map 4
                             Statistics: Num rows: 7433 Data size: 228226 Basic stats: COMPLETE Column stats: NONE
-                            Select Operator
-                              expressions: _col1 (type: double)
+                            Group By Operator
+                              aggregations: sum(_col1)
+                              mode: hash
                               outputColumnNames: _col0
-                              Statistics: Num rows: 7433 Data size: 228226 Basic stats: COMPLETE Column stats: NONE
-                              Group By Operator
-                                aggregations: sum(_col0)
-                                mode: hash
-                                outputColumnNames: _col0
+                              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                              Reduce Output Operator
+                                sort order: 
                                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                                Reduce Output Operator
-                                  sort order: 
-                                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                                  value expressions: _col0 (type: double)
+                                value expressions: _col0 (type: double)
             Local Work:
               Map Reduce Local Work
             Execution mode: vectorized

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out b/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
index 290db0d..ed1ba4b 100644
--- a/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
@@ -644,10 +644,10 @@ STAGE PLANS:
                   Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctimestamp1 (type: timestamp)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctimestamp1
                     Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0), max(_col0), count(_col0), count()
+                      aggregations: min(ctimestamp1), max(ctimestamp1), count(ctimestamp1), count()
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: NONE
@@ -725,10 +725,10 @@ STAGE PLANS:
                   Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctimestamp1 (type: timestamp)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctimestamp1
                     Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(ctimestamp1)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -815,10 +815,10 @@ STAGE PLANS:
                   Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctimestamp1 (type: timestamp)
-                    outputColumnNames: _col0
+                    outputColumnNames: ctimestamp1
                     Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: avg(_col0), variance(_col0), var_pop(_col0), var_samp(_col0), std(_col0), stddev(_col0), stddev_pop(_col0), stddev_samp(_col0)
+                      aggregations: avg(ctimestamp1), variance(ctimestamp1), var_pop(ctimestamp1), var_samp(ctimestamp1), std(ctimestamp1), stddev(ctimestamp1), stddev_pop(ctimestamp1), stddev_samp(ctimestamp1)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/stats_only_null.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_only_null.q.out b/ql/src/test/results/clientpositive/stats_only_null.q.out
index 55c5970..bbd32fe 100644
--- a/ql/src/test/results/clientpositive/stats_only_null.q.out
+++ b/ql/src/test/results/clientpositive/stats_only_null.q.out
@@ -85,10 +85,10 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: double), b (type: int), c (type: string), d (type: smallint)
-              outputColumnNames: _col0, _col1, _col2, _col3
+              outputColumnNames: a, b, c, d
               Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(), count(_col0), count(_col1), count(_col2), count(_col3)
+                aggregations: count(), count(a), count(b), count(c), count(d)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
@@ -135,10 +135,10 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: double), b (type: int), c (type: string), d (type: smallint)
-              outputColumnNames: _col0, _col1, _col2, _col3
+              outputColumnNames: a, b, c, d
               Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(), count(_col0), count(_col1), count(_col2), count(_col3)
+                aggregations: count(), count(a), count(b), count(c), count(d)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/stats_ppr_all.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_ppr_all.q.out b/ql/src/test/results/clientpositive/stats_ppr_all.q.out
index 5f6f5d4..7627f7a 100644
--- a/ql/src/test/results/clientpositive/stats_ppr_all.q.out
+++ b/ql/src/test/results/clientpositive/stats_ppr_all.q.out
@@ -77,10 +77,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: order_amount (type: float)
-              outputColumnNames: _col0
+              outputColumnNames: order_amount
               Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(order_amount)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -128,10 +128,10 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: PARTIAL
               Select Operator
                 expressions: order_amount (type: float)
-                outputColumnNames: _col0
+                outputColumnNames: order_amount
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: PARTIAL
                 Group By Operator
-                  aggregations: sum(_col0)
+                  aggregations: sum(order_amount)
                   mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: PARTIAL
@@ -176,10 +176,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: order_amount (type: float)
-              outputColumnNames: _col0
+              outputColumnNames: order_amount
               Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(order_amount)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -227,10 +227,10 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: order_amount (type: float)
-                outputColumnNames: _col0
+                outputColumnNames: order_amount
                 Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
-                  aggregations: sum(_col0)
+                  aggregations: sum(order_amount)
                   mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/subq_where_serialization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subq_where_serialization.q.out b/ql/src/test/results/clientpositive/subq_where_serialization.q.out
index c0b2a2d..2a60036 100644
--- a/ql/src/test/results/clientpositive/subq_where_serialization.q.out
+++ b/ql/src/test/results/clientpositive/subq_where_serialization.q.out
@@ -20,20 +20,16 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
+              Group By Operator
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/subquery_exists_having.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_exists_having.q.out b/ql/src/test/results/clientpositive/subquery_exists_having.q.out
index 14819c9..13877fc 100644
--- a/ql/src/test/results/clientpositive/subquery_exists_having.q.out
+++ b/ql/src/test/results/clientpositive/subquery_exists_having.q.out
@@ -35,22 +35,18 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -182,22 +178,18 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
           TableScan
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/subquery_in.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_in.q.out b/ql/src/test/results/clientpositive/subquery_in.q.out
index f82c799..f12af57 100644
--- a/ql/src/test/results/clientpositive/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/subquery_in.q.out
@@ -632,20 +632,16 @@ STAGE PLANS:
             Filter Operator
               predicate: ((key > '9') and value is not null) (type: boolean)
               Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                keys: key (type: string), value (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
@@ -886,20 +882,16 @@ STAGE PLANS:
             Filter Operator
               predicate: l_partkey is not null (type: boolean)
               Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: l_partkey (type: int)
+              Group By Operator
+                keys: l_partkey (type: int)
+                mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: int)
-                  mode: hash
-                  outputColumnNames: _col0
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
-                    Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: int)


[34/50] [abbrv] hive git commit: HIVE-11813: Avoid expensive AST tree conversion to String for expressions in WHERE clause in CBO (Jesus Camacho Rodriguez, reviewed by Hari Sankar Sivarama Subramaniyan)

Posted by xu...@apache.org.
HIVE-11813: Avoid expensive AST tree conversion to String for expressions in WHERE clause in CBO (Jesus Camacho Rodriguez, reviewed by Hari Sankar Sivarama Subramaniyan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/5a550cb4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/5a550cb4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/5a550cb4

Branch: refs/heads/spark
Commit: 5a550cb466664daaa2322aeaf5abf19f2b9d51c0
Parents: 03f46b2
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Sep 16 15:48:59 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Sep 16 15:48:59 2015 +0100

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/parse/CalcitePlanner.java     | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5a550cb4/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 8e992da..d5c747f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -1482,8 +1482,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
       return tableRel;
     }
 
-    private RelNode genFilterRelNode(ASTNode filterExpr, RelNode srcRel) throws SemanticException {
-      ExprNodeDesc filterCondn = genExprNodeDesc(filterExpr, relToHiveRR.get(srcRel));
+    private RelNode genFilterRelNode(ASTNode filterExpr, RelNode srcRel,
+            boolean useCaching) throws SemanticException {
+      ExprNodeDesc filterCondn = genExprNodeDesc(filterExpr, relToHiveRR.get(srcRel), useCaching);
       if (filterCondn instanceof ExprNodeConstantDesc
           && !filterCondn.getTypeString().equals(serdeConstants.BOOLEAN_TYPE_NAME)) {
         // queries like select * from t1 where 'foo';
@@ -1634,7 +1635,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
               subQuery.getJoinConditionAST());
           searchCond = subQuery.updateOuterQueryFilter(clonedSearchCond);
 
-          srcRel = genFilterRelNode(searchCond, srcRel);
+          srcRel = genFilterRelNode(searchCond, srcRel, forHavingClause);
 
           /*
            * For Not Exists and Not In, add a projection on top of the Left
@@ -1650,7 +1651,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         return srcRel;
       }
 
-      return genFilterRelNode(searchCond, srcRel);
+      return genFilterRelNode(searchCond, srcRel, forHavingClause);
     }
 
     private RelNode projectLeftOuterSide(RelNode srcRel, int numColumns) throws SemanticException {


[46/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/multiMapJoin2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/multiMapJoin2.q.out b/ql/src/test/results/clientpositive/multiMapJoin2.q.out
index 2cbef2e..46b717f 100644
--- a/ql/src/test/results/clientpositive/multiMapJoin2.q.out
+++ b/ql/src/test/results/clientpositive/multiMapJoin2.q.out
@@ -527,10 +527,10 @@ STAGE PLANS:
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -761,11 +761,11 @@ STAGE PLANS:
   Stage: Stage-17
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_0:$hdt$_0:$hdt$_1:y1 
+        $hdt$_0:$hdt$_1:y1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_0:$hdt$_0:$hdt$_1:y1 
+        $hdt$_0:$hdt$_1:y1 
           TableScan
             alias: y1
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -833,11 +833,11 @@ STAGE PLANS:
   Stage: Stage-15
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$INTNAME1 
+        $INTNAME1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$INTNAME1 
+        $INTNAME1 
           TableScan
             HashTable Sink Operator
               keys:
@@ -919,11 +919,11 @@ STAGE PLANS:
   Stage: Stage-16
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$INTNAME 
+        $INTNAME 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$INTNAME 
+        $INTNAME 
           TableScan
             HashTable Sink Operator
               keys:
@@ -997,11 +997,11 @@ STAGE PLANS:
   Stage: Stage-18
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:$hdt$_1:$hdt$_2:y1 
+        $hdt$_1:$hdt$_2:y1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:$hdt$_1:$hdt$_2:y1 
+        $hdt$_1:$hdt$_2:y1 
           TableScan
             alias: y1
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -1163,14 +1163,14 @@ STAGE PLANS:
   Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_0:$hdt$_0:$hdt$_1:y1 
+        $hdt$_0:$hdt$_1:y1 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_1:$hdt$_1:$hdt$_2:y1 
+        $hdt$_1:$hdt$_2:y1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_0:$hdt$_0:$hdt$_1:y1 
+        $hdt$_0:$hdt$_1:y1 
           TableScan
             alias: y1
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -1185,7 +1185,7 @@ STAGE PLANS:
                   keys:
                     0 _col0 (type: string)
                     1 _col0 (type: string)
-        $hdt$_0:$hdt$_1:$hdt$_1:$hdt$_2:y1 
+        $hdt$_1:$hdt$_2:y1 
           TableScan
             alias: y1
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -1459,20 +1459,16 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
+              Group By Operator
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)
@@ -1492,11 +1488,11 @@ STAGE PLANS:
   Stage: Stage-12
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$INTNAME1 
+        $INTNAME1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$INTNAME1 
+        $INTNAME1 
           TableScan
             HashTable Sink Operator
               keys:
@@ -1515,22 +1511,18 @@ STAGE PLANS:
                 1 _col0 (type: string)
               outputColumnNames: _col1
               Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: _col1 (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: _col1 (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+                File Output Operator
+                  compressed: false
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
       Local Work:
         Map Reduce Local Work
 
@@ -1582,11 +1574,11 @@ STAGE PLANS:
   Stage: Stage-13
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$INTNAME 
+        $INTNAME 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$INTNAME 
+        $INTNAME 
           TableScan
             HashTable Sink Operator
               keys:
@@ -1605,22 +1597,18 @@ STAGE PLANS:
                 1 _col0 (type: string)
               outputColumnNames: _col1
               Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: _col1 (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count()
+                keys: _col1 (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+                File Output Operator
+                  compressed: false
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
       Local Work:
         Map Reduce Local Work
 
@@ -1648,31 +1636,27 @@ STAGE PLANS:
             1 _col0 (type: string)
           outputColumnNames: _col1
           Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
-          Select Operator
-            expressions: _col1 (type: string)
-            outputColumnNames: _col0
+          Group By Operator
+            aggregations: count()
+            keys: _col1 (type: string)
+            mode: hash
+            outputColumnNames: _col0, _col1
             Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
-            Group By Operator
-              aggregations: count()
-              keys: _col0 (type: string)
-              mode: hash
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
-                table:
-                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-14
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_0:$hdt$_0:$hdt$_1:x1 
+        $hdt$_0:$hdt$_1:x1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_0:$hdt$_0:$hdt$_1:x1 
+        $hdt$_0:$hdt$_1:x1 
           TableScan
             alias: x1
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -1833,11 +1817,11 @@ STAGE PLANS:
   Stage: Stage-7
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_0:$hdt$_0:$hdt$_1:x1 
+        $hdt$_0:$hdt$_1:x1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_0:$hdt$_0:$hdt$_1:x1 
+        $hdt$_0:$hdt$_1:x1 
           TableScan
             alias: x1
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -1862,20 +1846,16 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
+              Group By Operator
+                keys: key (type: string)
+                mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
           TableScan
             alias: x2
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1924,24 +1904,20 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col1
                 Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: string)
-                  outputColumnNames: _col0
+                Mux Operator
                   Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                  Mux Operator
-                    Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                    Group By Operator
-                      aggregations: count()
-                      keys: _col0 (type: string)
-                      mode: complete
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                      File Output Operator
-                        compressed: false
-                        table:
-                            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                            serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+                  Group By Operator
+                    aggregations: count()
+                    keys: _col1 (type: string)
+                    mode: complete
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
           Group By Operator
             keys: KEY._col0 (type: string)
             mode: mergepartial
@@ -1957,24 +1933,20 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col1
                 Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: string)
-                  outputColumnNames: _col0
+                Mux Operator
                   Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                  Mux Operator
-                    Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
-                    Group By Operator
-                      aggregations: count()
-                      keys: _col0 (type: string)
-                      mode: complete
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                      File Output Operator
-                        compressed: false
-                        table:
-                            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                            serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+                  Group By Operator
+                    aggregations: count()
+                    keys: _col1 (type: string)
+                    mode: complete
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-3
     Map Reduce
@@ -2228,11 +2200,11 @@ STAGE PLANS:
   Stage: Stage-15
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        null-subquery1:$hdt$_0-subquery1:$hdt$_1:$hdt$_1:$hdt$_1:a 
+        null-subquery1:$hdt$_0-subquery1:$hdt$_1:$hdt$_1:a 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        null-subquery1:$hdt$_0-subquery1:$hdt$_1:$hdt$_1:$hdt$_1:a 
+        null-subquery1:$hdt$_0-subquery1:$hdt$_1:$hdt$_1:a 
           TableScan
             alias: a
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -2380,11 +2352,11 @@ STAGE PLANS:
   Stage: Stage-16
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        null-subquery2:$hdt$_0-subquery2:$hdt$_1:$hdt$_1:$hdt$_1:a 
+        null-subquery2:$hdt$_0-subquery2:$hdt$_1:$hdt$_1:a 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        null-subquery2:$hdt$_0-subquery2:$hdt$_1:$hdt$_1:$hdt$_1:a 
+        null-subquery2:$hdt$_0-subquery2:$hdt$_1:$hdt$_1:a 
           TableScan
             alias: a
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/nonblock_op_deduplicate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/nonblock_op_deduplicate.q.out b/ql/src/test/results/clientpositive/nonblock_op_deduplicate.q.out
index feadd5a..9a24ad8 100644
--- a/ql/src/test/results/clientpositive/nonblock_op_deduplicate.q.out
+++ b/ql/src/test/results/clientpositive/nonblock_op_deduplicate.q.out
@@ -37,7 +37,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: -- This test query is introduced for HIVE-4968.
 -- First, we do not convert the join to MapJoin.
 EXPLAIN
@@ -140,7 +140,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT tmp4.key as key, tmp4.value as value, tmp4.count as count
 FROM (SELECT tmp2.key as key, tmp2.value as value, tmp3.count as count
       FROM (SELECT *
@@ -188,7 +188,7 @@ POSTHOOK: Input: default@src1
 406	val_406	25
 66	val_66	25
 98	val_98	25
-Warning: Map Join MAPJOIN[18][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[17][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
 PREHOOK: query: -- Then, we convert the join to MapJoin.
 EXPLAIN
 SELECT tmp4.key as key, tmp4.value as value, tmp4.count as count
@@ -296,7 +296,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Map Join MAPJOIN[18][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[17][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
 PREHOOK: query: SELECT tmp4.key as key, tmp4.value as value, tmp4.count as count
 FROM (SELECT tmp2.key as key, tmp2.value as value, tmp3.count as count
       FROM (SELECT *

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/nonmr_fetch.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/nonmr_fetch.q.out b/ql/src/test/results/clientpositive/nonmr_fetch.q.out
index a4ce905..9652d7e 100644
--- a/ql/src/test/results/clientpositive/nonmr_fetch.q.out
+++ b/ql/src/test/results/clientpositive/nonmr_fetch.q.out
@@ -974,11 +974,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col1)
-                keys: _col0 (type: string)
+                aggregations: count(value)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1028,10 +1028,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1081,10 +1081,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/partition_multilevels.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/partition_multilevels.q.out b/ql/src/test/results/clientpositive/partition_multilevels.q.out
index 31862e1..c1c8778 100644
--- a/ql/src/test/results/clientpositive/partition_multilevels.q.out
+++ b/ql/src/test/results/clientpositive/partition_multilevels.q.out
@@ -991,11 +991,11 @@ STAGE PLANS:
             Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: '2222' (type: string), level2 (type: string), level3 (type: string)
-              outputColumnNames: _col0, _col1, _col2
+              outputColumnNames: level1, level2, level3
               Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+                keys: level1 (type: string), level2 (type: string), level3 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE
@@ -1579,11 +1579,11 @@ STAGE PLANS:
             Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: '2222' (type: string), level2 (type: string), level3 (type: string)
-              outputColumnNames: _col0, _col1, _col2
+              outputColumnNames: level1, level2, level3
               Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+                keys: level1 (type: string), level2 (type: string), level3 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/ppd_gby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ppd_gby.q.out b/ql/src/test/results/clientpositive/ppd_gby.q.out
index 87cb907..6f8ee53 100644
--- a/ql/src/test/results/clientpositive/ppd_gby.q.out
+++ b/ql/src/test/results/clientpositive/ppd_gby.q.out
@@ -33,11 +33,11 @@ STAGE PLANS:
                   Statistics: Num rows: 18 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: _col1 (type: string), _col0 (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: _col1, _col0
                     Statistics: Num rows: 18 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: count(_col0)
+                      keys: _col1 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 18 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -246,11 +246,11 @@ STAGE PLANS:
               Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: value (type: string), key (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: _col1, _col0
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: count(_col1)
-                  keys: _col0 (type: string)
+                  aggregations: count(_col0)
+                  keys: _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/ppd_gby2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ppd_gby2.q.out b/ql/src/test/results/clientpositive/ppd_gby2.q.out
index bc00149..5fbe70f 100644
--- a/ql/src/test/results/clientpositive/ppd_gby2.q.out
+++ b/ql/src/test/results/clientpositive/ppd_gby2.q.out
@@ -36,11 +36,11 @@ STAGE PLANS:
                   Statistics: Num rows: 18 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: _col1 (type: string), _col0 (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: _col1, _col0
                     Statistics: Num rows: 18 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: count(_col0)
+                      keys: _col1 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 18 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -60,22 +60,18 @@ STAGE PLANS:
           Filter Operator
             predicate: ((_col1 > 30) or (_col0 < 'val_400')) (type: boolean)
             Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: _col1 (type: bigint), _col0 (type: string)
+            Group By Operator
+              aggregations: max(_col0)
+              keys: _col1 (type: bigint)
+              mode: hash
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE
-              Group By Operator
-                aggregations: max(_col1)
-                keys: _col0 (type: bigint)
-                mode: hash
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-2
     Map Reduce
@@ -164,11 +160,11 @@ STAGE PLANS:
               Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: value (type: string), key (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: _col1, _col0
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: count(_col1)
-                  keys: _col0 (type: string)
+                  aggregations: count(_col0)
+                  keys: _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -188,22 +184,18 @@ STAGE PLANS:
           Filter Operator
             predicate: ((_col1 > 30) or (_col0 < 'val_400')) (type: boolean)
             Statistics: Num rows: 18 Data size: 190 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: _col1 (type: bigint), _col0 (type: string)
+            Group By Operator
+              aggregations: max(_col0)
+              keys: _col1 (type: bigint)
+              mode: hash
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 18 Data size: 190 Basic stats: COMPLETE Column stats: NONE
-              Group By Operator
-                aggregations: max(_col1)
-                keys: _col0 (type: bigint)
-                mode: hash
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 18 Data size: 190 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-2
     Map Reduce

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/ppd_join_filter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ppd_join_filter.q.out b/ql/src/test/results/clientpositive/ppd_join_filter.q.out
index e99986f..1781fe9 100644
--- a/ql/src/test/results/clientpositive/ppd_join_filter.q.out
+++ b/ql/src/test/results/clientpositive/ppd_join_filter.q.out
@@ -136,24 +136,20 @@ STAGE PLANS:
               isSamplingPred: false
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: min(key)
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: min(_col0)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    tag: -1
-                    value expressions: _col1 (type: string)
-                    auto parallelism: false
+                  tag: -1
+                  value expressions: _col1 (type: string)
+                  auto parallelism: false
       Path -> Alias:
 #### A masked pattern was here ####
       Path -> Partition:
@@ -202,7 +198,7 @@ STAGE PLANS:
               name: default.src
             name: default.src
       Truncated Path -> Alias:
-        /src [$hdt$_1:$hdt$_1:a]
+        /src [$hdt$_1:a]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -561,24 +557,20 @@ STAGE PLANS:
               isSamplingPred: false
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: min(key)
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: min(_col0)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    tag: -1
-                    value expressions: _col1 (type: string)
-                    auto parallelism: false
+                  tag: -1
+                  value expressions: _col1 (type: string)
+                  auto parallelism: false
       Path -> Alias:
 #### A masked pattern was here ####
       Path -> Partition:
@@ -627,7 +619,7 @@ STAGE PLANS:
               name: default.src
             name: default.src
       Truncated Path -> Alias:
-        /src [$hdt$_1:$hdt$_1:a]
+        /src [$hdt$_1:a]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -976,11 +968,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: min(_col0)
-                keys: _col0 (type: string)
+                aggregations: min(key)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1040,7 +1032,7 @@ STAGE PLANS:
               name: default.src
             name: default.src
       Truncated Path -> Alias:
-        /src [$hdt$_1:$hdt$_1:a]
+        /src [$hdt$_1:a]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -1395,24 +1387,20 @@ STAGE PLANS:
               isSamplingPred: false
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: min(key)
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: min(_col0)
-                  keys: _col0 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    tag: -1
-                    value expressions: _col1 (type: string)
-                    auto parallelism: false
+                  tag: -1
+                  value expressions: _col1 (type: string)
+                  auto parallelism: false
       Path -> Alias:
 #### A masked pattern was here ####
       Path -> Partition:
@@ -1461,7 +1449,7 @@ STAGE PLANS:
               name: default.src
             name: default.src
       Truncated Path -> Alias:
-        /src [$hdt$_1:$hdt$_1:a]
+        /src [$hdt$_1:a]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx_cbo_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx_cbo_1.q.out b/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx_cbo_1.q.out
index 7d1aff2..6f00df9 100644
--- a/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx_cbo_1.q.out
+++ b/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx_cbo_1.q.out
@@ -92,11 +92,11 @@ STAGE PLANS:
             Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: l_shipdate (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: l_shipdate
               Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
-                keys: _col0 (type: string)
+                aggregations: count(l_shipdate)
+                keys: l_shipdate (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
@@ -253,15 +253,15 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: $hdt$_0:default__lineitem_ix_lineitem_ix_lshipdate_idx__
+            alias: default__lineitem_ix_lineitem_ix_lshipdate_idx__
             Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint)
-              outputColumnNames: _col0, _count_of_l_shipdate
+              outputColumnNames: l_shipdate, _count_of_l_shipdate
               Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: sum(_count_of_l_shipdate)
-                keys: _col0 (type: string)
+                keys: l_shipdate (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
@@ -900,15 +900,15 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: null-subquery1:$hdt$_0-subquery1:$hdt$_0:default__lineitem_ix_lineitem_ix_lshipdate_idx__
+            alias: null-subquery1:$hdt$_0-subquery1:default__lineitem_ix_lineitem_ix_lshipdate_idx__
             Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint)
-              outputColumnNames: _col0, _count_of_l_shipdate
+              outputColumnNames: l_shipdate, _count_of_l_shipdate
               Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: sum(_count_of_l_shipdate)
-                keys: _col0 (type: string)
+                keys: l_shipdate (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE
@@ -1015,11 +1015,11 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: 1 (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
-                  aggregations: count(_col0)
-                  keys: _col0 (type: int)
+                  aggregations: count(key)
+                  keys: key (type: int)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1063,15 +1063,15 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: $hdt$_0:default__tbl_tbl_key_idx__
+            alias: default__tbl_tbl_key_idx__
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
               expressions: key (type: int), _count_of_key (type: bigint)
-              outputColumnNames: _col0, _count_of_key
+              outputColumnNames: key, _count_of_key
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
                 aggregations: sum(_count_of_key)
-                keys: _col0 (type: int)
+                keys: key (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1161,11 +1161,11 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: $hdt$_0:default__tbl_tbl_key_idx__
+            alias: default__tbl_tbl_key_idx__
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
               expressions: key (type: int), _count_of_key (type: bigint)
-              outputColumnNames: _col0, _count_of_key
+              outputColumnNames: key, _count_of_key
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
                 aggregations: sum(_count_of_key)
@@ -1213,10 +1213,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
               expressions: key (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                keys: _col0 (type: int)
+                keys: key (type: int)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1261,11 +1261,11 @@ STAGE PLANS:
             alias: tbl
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
-              expressions: value (type: int), key (type: int)
-              outputColumnNames: _col0, _col1
+              expressions: key (type: int), value (type: int)
+              outputColumnNames: key, value
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                keys: _col0 (type: int), _col1 (type: int)
+                keys: key (type: int), value (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1281,7 +1281,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
-            expressions: _col1 (type: int)
+            expressions: _col0 (type: int)
             outputColumnNames: _col0
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             File Output Operator
@@ -1318,10 +1318,10 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: 3 (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
-                  keys: _col0 (type: int)
+                  keys: key (type: int)
                   mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1370,10 +1370,10 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
-                  keys: _col0 (type: int)
+                  keys: key (type: int)
                   mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1471,11 +1471,11 @@ STAGE PLANS:
             alias: tbl
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
-              expressions: value (type: int), key (type: int)
-              outputColumnNames: _col0, _col1
+              expressions: key (type: int), value (type: int)
+              outputColumnNames: key, value
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                keys: _col0 (type: int), _col1 (type: int)
+                keys: key (type: int), value (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1490,17 +1490,13 @@ STAGE PLANS:
           mode: mergepartial
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-          Select Operator
-            expressions: _col1 (type: int), _col0 (type: int)
-            outputColumnNames: _col0, _col1
+          File Output Operator
+            compressed: false
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -1528,10 +1524,10 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: key (type: int), 1 (type: int)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: key, value
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
-                  keys: _col0 (type: int), _col1 (type: int)
+                  keys: key (type: int), value (type: int)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1577,10 +1573,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
               expressions: key (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                keys: _col0 (type: int)
+                keys: key (type: int)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1626,10 +1622,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
               expressions: key (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                keys: _col0 (type: int)
+                keys: key (type: int)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1675,10 +1671,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
               expressions: key (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                keys: _col0 (type: int)
+                keys: key (type: int)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1724,10 +1720,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
               expressions: key (type: int), value (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                keys: _col0 (type: int), _col1 (type: int)
+                keys: key (type: int), value (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1776,10 +1772,10 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: key (type: int), 2 (type: int)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: key, value
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
-                  keys: _col0 (type: int), _col1 (type: int)
+                  keys: key (type: int), value (type: int)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1828,10 +1824,10 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: 3 (type: int), 2 (type: int)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: key, value
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
-                  keys: _col0 (type: int), _col1 (type: int)
+                  keys: key (type: int), value (type: int)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1878,20 +1874,16 @@ STAGE PLANS:
             Filter Operator
               predicate: (value = key) (type: boolean)
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-              Select Operator
-                expressions: key (type: int), value (type: int)
+              Group By Operator
+                keys: key (type: int), value (type: int)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: int), _col1 (type: int)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: int), _col1 (type: int)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int), _col1 (type: int)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
-                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: int), KEY._col1 (type: int)
@@ -2033,10 +2025,10 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: key (type: int), 2 (type: int)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: key, value
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
-                  keys: _col0 (type: int), _col1 (type: int)
+                  keys: key (type: int), value (type: int)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -2168,22 +2160,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (key < 10) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: int)
-                outputColumnNames: _col0
+              Group By Operator
+                aggregations: count(key)
+                keys: key (type: int)
+                mode: hash
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count(_col0)
-                  keys: _col0 (type: int)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: bigint)
+                  value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
@@ -2260,18 +2248,18 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: $hdt$_0:default__tblpart_tbl_part_index__
+            alias: default__tblpart_tbl_part_index__
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key < 10) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: int), _count_of_key (type: bigint)
-                outputColumnNames: _col0, _count_of_key
+                outputColumnNames: key, _count_of_key
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   aggregations: sum(_count_of_key)
-                  keys: _col0 (type: int)
+                  keys: key (type: int)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
@@ -2369,11 +2357,11 @@ STAGE PLANS:
             Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
-                keys: _col0 (type: int)
+                aggregations: count(key)
+                keys: key (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE
@@ -2453,15 +2441,15 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: $hdt$_0:default__tbl_tbl_key_idx__
+            alias: default__tbl_tbl_key_idx__
             Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: int), _count_of_key (type: bigint)
-              outputColumnNames: _col0, _count_of_key
+              outputColumnNames: key, _count_of_key
               Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: sum(_count_of_key)
-                keys: _col0 (type: int)
+                keys: key (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx_cbo_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx_cbo_2.q.out b/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx_cbo_2.q.out
index 3ee2e0f..667c683 100644
--- a/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx_cbo_2.q.out
+++ b/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx_cbo_2.q.out
@@ -176,11 +176,11 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: $hdt$_0:default__lineitem_ix_lineitem_ix_l_orderkey_idx__
+            alias: default__lineitem_ix_lineitem_ix_l_orderkey_idx__
             Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint)
-              outputColumnNames: _col0, _count_of_l_orderkey
+              outputColumnNames: l_orderkey, _count_of_l_orderkey
               Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: sum(_count_of_l_orderkey)
@@ -247,11 +247,11 @@ STAGE PLANS:
             Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: l_orderkey (type: int), l_partkey (type: int)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: l_orderkey, l_partkey
               Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0), count(_col1)
-                keys: _col0 (type: int), _col1 (type: int)
+                aggregations: count(l_orderkey), count(l_partkey)
+                keys: l_orderkey (type: int), l_partkey (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
@@ -428,11 +428,11 @@ STAGE PLANS:
               Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: 7 (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: l_orderkey
                 Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: count(_col0)
-                  keys: _col0 (type: int)
+                  aggregations: count(l_orderkey)
+                  keys: l_orderkey (type: int)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE
@@ -1131,11 +1131,11 @@ STAGE PLANS:
             Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: l_orderkey (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: l_orderkey
               Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0), sum(_col0)
-                keys: _col0 (type: int)
+                aggregations: count(l_orderkey), sum(l_orderkey)
+                keys: l_orderkey (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE
@@ -1223,15 +1223,15 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: $hdt$_0:default__lineitem_ix_lineitem_ix_l_orderkey_idx__
+            alias: default__lineitem_ix_lineitem_ix_l_orderkey_idx__
             Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint)
-              outputColumnNames: _col0, _count_of_l_orderkey
+              outputColumnNames: l_orderkey, _count_of_l_orderkey
               Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: sum(_count_of_l_orderkey)
-                keys: _col0 (type: int)
+                keys: l_orderkey (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE
@@ -1812,11 +1812,11 @@ STAGE PLANS:
           Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col1 (type: bigint)
-            outputColumnNames: _col0
+            outputColumnNames: _col1
             Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE
             Group By Operator
-              aggregations: count(_col0)
-              keys: _col0 (type: bigint)
+              aggregations: count(_col1)
+              keys: _col1 (type: bigint)
               mode: hash
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE
@@ -1907,18 +1907,18 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: $hdt$_0:$hdt$_0:default__lineitem_ix_lineitem_ix_l_orderkey_idx__
+            alias: $hdt$_0:default__lineitem_ix_lineitem_ix_l_orderkey_idx__
             Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (l_orderkey < 7) (type: boolean)
               Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint)
-                outputColumnNames: _col0, _count_of_l_orderkey
+                outputColumnNames: l_orderkey, _count_of_l_orderkey
                 Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   aggregations: sum(_count_of_l_orderkey)
-                  keys: _col0 (type: int)
+                  keys: l_orderkey (type: int)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE
@@ -1937,11 +1937,11 @@ STAGE PLANS:
           Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col1 (type: bigint)
-            outputColumnNames: _col0
+            outputColumnNames: _col1
             Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE
             Group By Operator
-              aggregations: count(_col0)
-              keys: _col0 (type: bigint)
+              aggregations: count(_col1)
+              keys: _col1 (type: bigint)
               mode: hash
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE
@@ -3404,18 +3404,18 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: $hdt$_0-subquery1:$hdt$_0-subquery1:$hdt$_0:default__lineitem_ix_lineitem_ix_l_orderkey_idx__
+            alias: $hdt$_0-subquery1:$hdt$_0-subquery1:default__lineitem_ix_lineitem_ix_l_orderkey_idx__
             Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (l_orderkey < 7) (type: boolean)
               Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint)
-                outputColumnNames: _col0, _count_of_l_orderkey
+                outputColumnNames: l_orderkey, _count_of_l_orderkey
                 Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   aggregations: sum(_count_of_l_orderkey)
-                  keys: _col0 (type: int)
+                  keys: l_orderkey (type: int)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE
@@ -3495,18 +3495,18 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: $hdt$_0-subquery2:$hdt$_0-subquery2:$hdt$_0:default__lineitem_ix_lineitem_ix_l_partkey_idx__
+            alias: $hdt$_0-subquery2:$hdt$_0-subquery2:default__lineitem_ix_lineitem_ix_l_partkey_idx__
             Statistics: Num rows: 100 Data size: 8937 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (l_partkey < 10) (type: boolean)
               Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: l_partkey (type: int), _count_of_l_partkey (type: bigint)
-                outputColumnNames: _col0, _count_of_l_partkey
+                outputColumnNames: l_partkey, _count_of_l_partkey
                 Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   aggregations: sum(_count_of_l_partkey)
-                  keys: _col0 (type: int)
+                  keys: l_partkey (type: int)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE
@@ -3633,20 +3633,16 @@ STAGE PLANS:
             Filter Operator
               predicate: (l_orderkey < 7) (type: boolean)
               Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: l_orderkey (type: int)
+              Group By Operator
+                keys: l_orderkey (type: int)
+                mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: int)
-                  mode: hash
-                  outputColumnNames: _col0
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
-                    Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: int)
@@ -3717,20 +3713,16 @@ STAGE PLANS:
             Filter Operator
               predicate: (l_partkey < 10) (type: boolean)
               Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: l_partkey (type: int)
+              Group By Operator
+                keys: l_partkey (type: int)
+                mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: int)
-                  mode: hash
-                  outputColumnNames: _col0
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
-                    Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: int)

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/reduce_deduplicate_extended.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/reduce_deduplicate_extended.q.out b/ql/src/test/results/clientpositive/reduce_deduplicate_extended.q.out
index df16ab2..e3ebee7 100644
--- a/ql/src/test/results/clientpositive/reduce_deduplicate_extended.q.out
+++ b/ql/src/test/results/clientpositive/reduce_deduplicate_extended.q.out
@@ -184,11 +184,11 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
-                keys: _col0 (type: string)
+                aggregations: sum(key)
+                keys: key (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -430,10 +430,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -488,10 +488,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -2682,12 +2682,12 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: string)
+                key expressions: key (type: string)
                 sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+                Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
@@ -2914,12 +2914,12 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: string), _col1 (type: string)
+                key expressions: key (type: string), value (type: string)
                 sort order: ++
-                Map-reduce partition columns: _col0 (type: string)
+                Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
@@ -2967,12 +2967,12 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
-                key expressions: _col0 (type: string), _col1 (type: string)
+                key expressions: key (type: string), value (type: string)
                 sort order: ++
-                Map-reduce partition columns: _col0 (type: string)
+                Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/selectDistinctStar.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/selectDistinctStar.q.out b/ql/src/test/results/clientpositive/selectDistinctStar.q.out
index abd6795..3f44580 100644
--- a/ql/src/test/results/clientpositive/selectDistinctStar.q.out
+++ b/ql/src/test/results/clientpositive/selectDistinctStar.q.out
@@ -19,10 +19,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -707,20 +707,16 @@ STAGE PLANS:
             Filter Operator
               predicate: (key < '3') (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                keys: key (type: string), value (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
@@ -2449,10 +2445,10 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: key, value
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: string), _col1 (type: string)
+                keys: key (type: string), value (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -3133,20 +3129,16 @@ STAGE PLANS:
             Filter Operator
               predicate: (key < '3') (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
+              Group By Operator
+                keys: key (type: string), value (type: string)
+                mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string)
-                  mode: hash
-                  outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string)


[19/50] [abbrv] hive git commit: HIVE-11763: Use * instead of sum(hash(*)) on Parquet predicate (PPD) integration tests (Sergio Pena, reviewed by Ferdinand Xu)

Posted by xu...@apache.org.
HIVE-11763: Use * instead of sum(hash(*)) on Parquet predicate (PPD) integration tests (Sergio Pena, reviewed by Ferdinand Xu)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/66fb9601
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/66fb9601
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/66fb9601

Branch: refs/heads/spark
Commit: 66fb9601dc1195518f2cc764851823c34c9d7dfe
Parents: bbb9129
Author: Sergio Pena <se...@cloudera.com>
Authored: Sun Sep 13 00:20:27 2015 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Sun Sep 13 00:20:27 2015 -0500

----------------------------------------------------------------------
 .../clientpositive/parquet_ppd_boolean.q        |  42 +-
 .../queries/clientpositive/parquet_ppd_char.q   |  46 +-
 .../queries/clientpositive/parquet_ppd_date.q   |  64 +-
 .../clientpositive/parquet_ppd_decimal.q        | 106 ++--
 .../clientpositive/parquet_ppd_timestamp.q      |  62 +-
 .../clientpositive/parquet_ppd_varchar.q        |  46 +-
 .../clientpositive/parquet_predicate_pushdown.q |  20 +-
 .../clientpositive/parquet_ppd_boolean.q.out    | 194 ++++--
 .../clientpositive/parquet_ppd_char.q.out       | 224 ++++---
 .../clientpositive/parquet_ppd_date.q.out       | 324 +++++++---
 .../clientpositive/parquet_ppd_decimal.q.out    | 594 ++++++++++++++-----
 .../clientpositive/parquet_ppd_timestamp.q.out  | 314 +++++++---
 .../clientpositive/parquet_ppd_varchar.q.out    | 224 ++++---
 .../parquet_predicate_pushdown.q.out            | 118 ++--
 14 files changed, 1584 insertions(+), 794 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/queries/clientpositive/parquet_ppd_boolean.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_ppd_boolean.q b/ql/src/test/queries/clientpositive/parquet_ppd_boolean.q
index 05c6c50..a7848b4 100644
--- a/ql/src/test/queries/clientpositive/parquet_ppd_boolean.q
+++ b/ql/src/test/queries/clientpositive/parquet_ppd_boolean.q
@@ -4,32 +4,32 @@ SET mapred.max.split.size=5000;
 
 create table newtypestbl(c char(10), v varchar(10), d decimal(5,3), b boolean) stored as parquet;
 
-insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, true from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, false from src src2) uniontbl;
+insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, true from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, false from src src2 limit 10) uniontbl;
 
 SET hive.optimize.ppd=true;
 SET hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where b=true;
-select sum(hash(*)) from newtypestbl where b!=true;
-select sum(hash(*)) from newtypestbl where b<true;
-select sum(hash(*)) from newtypestbl where b>true;
-select sum(hash(*)) from newtypestbl where b<=true;
+select * from newtypestbl where b=true;
+select * from newtypestbl where b!=true;
+select * from newtypestbl where b<true;
+select * from newtypestbl where b>true;
+select * from newtypestbl where b<=true;
 
-select sum(hash(*)) from newtypestbl where b=false;
-select sum(hash(*)) from newtypestbl where b!=false;
-select sum(hash(*)) from newtypestbl where b<false;
-select sum(hash(*)) from newtypestbl where b>false;
-select sum(hash(*)) from newtypestbl where b<=false;
+select * from newtypestbl where b=false;
+select * from newtypestbl where b!=false;
+select * from newtypestbl where b<false;
+select * from newtypestbl where b>false;
+select * from newtypestbl where b<=false;
 
 
 SET hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where b=true;
-select sum(hash(*)) from newtypestbl where b!=true;
-select sum(hash(*)) from newtypestbl where b<true;
-select sum(hash(*)) from newtypestbl where b>true;
-select sum(hash(*)) from newtypestbl where b<=true;
+select * from newtypestbl where b=true;
+select * from newtypestbl where b!=true;
+select * from newtypestbl where b<true;
+select * from newtypestbl where b>true;
+select * from newtypestbl where b<=true;
 
-select sum(hash(*)) from newtypestbl where b=false;
-select sum(hash(*)) from newtypestbl where b!=false;
-select sum(hash(*)) from newtypestbl where b<false;
-select sum(hash(*)) from newtypestbl where b>false;
-select sum(hash(*)) from newtypestbl where b<=false;
\ No newline at end of file
+select * from newtypestbl where b=false;
+select * from newtypestbl where b!=false;
+select * from newtypestbl where b<false;
+select * from newtypestbl where b>false;
+select * from newtypestbl where b<=false;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/queries/clientpositive/parquet_ppd_char.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_ppd_char.q b/ql/src/test/queries/clientpositive/parquet_ppd_char.q
index b01612c..dcad622 100644
--- a/ql/src/test/queries/clientpositive/parquet_ppd_char.q
+++ b/ql/src/test/queries/clientpositive/parquet_ppd_char.q
@@ -5,72 +5,72 @@ SET mapred.max.split.size=5000;
 
 create table newtypestbl(c char(10), v varchar(10), d decimal(5,3), da date) stored as parquet;
 
-insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl;
+insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl;
 
 set hive.optimize.index.filter=false;
 
 -- char data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where c="apple";
+select * from newtypestbl where c="apple";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c="apple";
+select * from newtypestbl where c="apple";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where c!="apple";
+select * from newtypestbl where c!="apple";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c!="apple";
+select * from newtypestbl where c!="apple";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where c<"hello";
+select * from newtypestbl where c<"hello";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c<"hello";
+select * from newtypestbl where c<"hello";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where c<="hello";
+select * from newtypestbl where c<="hello";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c<="hello";
+select * from newtypestbl where c<="hello";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where c="apple ";
+select * from newtypestbl where c="apple ";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c="apple ";
+select * from newtypestbl where c="apple ";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where c in ("apple", "carrot");
+select * from newtypestbl where c in ("apple", "carrot");
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c in ("apple", "carrot");
+select * from newtypestbl where c in ("apple", "carrot");
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where c in ("apple", "hello");
+select * from newtypestbl where c in ("apple", "hello");
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c in ("apple", "hello");
+select * from newtypestbl where c in ("apple", "hello");
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where c in ("carrot");
+select * from newtypestbl where c in ("carrot");
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c in ("carrot");
+select * from newtypestbl where c in ("carrot");
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where c between "apple" and "carrot";
+select * from newtypestbl where c between "apple" and "carrot";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c between "apple" and "carrot";
+select * from newtypestbl where c between "apple" and "carrot";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where c between "apple" and "zombie";
+select * from newtypestbl where c between "apple" and "zombie";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c between "apple" and "zombie";
+select * from newtypestbl where c between "apple" and "zombie";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where c between "carrot" and "carrot1";
+select * from newtypestbl where c between "carrot" and "carrot1";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where c between "carrot" and "carrot1";
\ No newline at end of file
+select * from newtypestbl where c between "carrot" and "carrot1";
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/queries/clientpositive/parquet_ppd_date.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_ppd_date.q b/ql/src/test/queries/clientpositive/parquet_ppd_date.q
index a18a9cf..a05d358 100644
--- a/ql/src/test/queries/clientpositive/parquet_ppd_date.q
+++ b/ql/src/test/queries/clientpositive/parquet_ppd_date.q
@@ -5,97 +5,97 @@ SET mapred.max.split.size=5000;
 
 create table newtypestbl(c char(10), v varchar(10), d decimal(5,3), da date) stored as parquet;
 
-insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl;
+insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl;
 
 -- date data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where da='1970-02-20';
+select * from newtypestbl where da='1970-02-20';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da='1970-02-20';
+select * from newtypestbl where da='1970-02-20';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da= date '1970-02-20';
+select * from newtypestbl where da= date '1970-02-20';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as date);
+select * from newtypestbl where da=cast('1970-02-20' as date);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as date);
+select * from newtypestbl where da=cast('1970-02-20' as date);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as varchar(20));
+select * from newtypestbl where da=cast('1970-02-20' as varchar(20));
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as varchar(20));
+select * from newtypestbl where da=cast('1970-02-20' as varchar(20));
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da!='1970-02-20';
+select * from newtypestbl where da!='1970-02-20';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da!='1970-02-20';
+select * from newtypestbl where da!='1970-02-20';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da<'1970-02-27';
+select * from newtypestbl where da<'1970-02-27';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da<'1970-02-27';
+select * from newtypestbl where da<'1970-02-27';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da<'1970-02-29';
+select * from newtypestbl where da<'1970-02-29';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da<'1970-02-29';
+select * from newtypestbl where da<'1970-02-29';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da<'1970-02-15';
+select * from newtypestbl where da<'1970-02-15';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da<'1970-02-15';
+select * from newtypestbl where da<'1970-02-15';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da<='1970-02-20';
+select * from newtypestbl where da<='1970-02-20';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da<='1970-02-20';
+select * from newtypestbl where da<='1970-02-20';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da<='1970-02-27';
+select * from newtypestbl where da<='1970-02-27';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da<='1970-02-27';
+select * from newtypestbl where da<='1970-02-27';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date));
+select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date));
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date));
+select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date));
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date));
+select * from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date));
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date));
+select * from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date));
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date));
+select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date));
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date));
+select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date));
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-22';
+select * from newtypestbl where da between '1970-02-19' and '1970-02-22';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-22';
+select * from newtypestbl where da between '1970-02-19' and '1970-02-22';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-28';
+select * from newtypestbl where da between '1970-02-19' and '1970-02-28';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-28';
+select * from newtypestbl where da between '1970-02-19' and '1970-02-28';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where da between '1970-02-18' and '1970-02-19';
+select * from newtypestbl where da between '1970-02-18' and '1970-02-19';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where da between '1970-02-18' and '1970-02-19';
+select * from newtypestbl where da between '1970-02-18' and '1970-02-19';

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/queries/clientpositive/parquet_ppd_decimal.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_ppd_decimal.q b/ql/src/test/queries/clientpositive/parquet_ppd_decimal.q
index 679164b..cf7cba0 100644
--- a/ql/src/test/queries/clientpositive/parquet_ppd_decimal.q
+++ b/ql/src/test/queries/clientpositive/parquet_ppd_decimal.q
@@ -4,160 +4,160 @@ SET mapred.max.split.size=5000;
 
 create table newtypestbl(c char(10), v varchar(10), d decimal(5,3), da date) stored as parquet;
 
-insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl;
+insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl;
 
 -- decimal data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where d=0.22;
+select * from newtypestbl where d=0.22;
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d=0.22;
+select * from newtypestbl where d=0.22;
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d='0.22';
+select * from newtypestbl where d='0.22';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d='0.22';
+select * from newtypestbl where d='0.22';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d=cast('0.22' as float);
+select * from newtypestbl where d=cast('0.22' as float);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d=cast('0.22' as float);
+select * from newtypestbl where d=cast('0.22' as float);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d!=0.22;
+select * from newtypestbl where d!=0.22;
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d!=0.22;
+select * from newtypestbl where d!=0.22;
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d!='0.22';
+select * from newtypestbl where d!='0.22';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d!='0.22';
+select * from newtypestbl where d!='0.22';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d!=cast('0.22' as float);
+select * from newtypestbl where d!=cast('0.22' as float);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d!=cast('0.22' as float);
+select * from newtypestbl where d!=cast('0.22' as float);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d<11.22;
+select * from newtypestbl where d<11.22;
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d<11.22;
+select * from newtypestbl where d<11.22;
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d<'11.22';
+select * from newtypestbl where d<'11.22';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d<'11.22';
+select * from newtypestbl where d<'11.22';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d<cast('11.22' as float);
+select * from newtypestbl where d<cast('11.22' as float);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d<cast('11.22' as float);
+select * from newtypestbl where d<cast('11.22' as float);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d<1;
+select * from newtypestbl where d<1;
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d<1;
+select * from newtypestbl where d<1;
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d<=11.22;
+select * from newtypestbl where d<=11.22;
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d<=11.22;
+select * from newtypestbl where d<=11.22;
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d<='11.22';
+select * from newtypestbl where d<='11.22';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d<='11.22';
+select * from newtypestbl where d<='11.22';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d<=cast('11.22' as float);
+select * from newtypestbl where d<=cast('11.22' as float);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d<=cast('11.22' as float);
+select * from newtypestbl where d<=cast('11.22' as float);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d<=cast('11.22' as decimal);
+select * from newtypestbl where d<=cast('11.22' as decimal);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d<=cast('11.22' as decimal);
+select * from newtypestbl where d<=cast('11.22' as decimal);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d<=11.22BD;
+select * from newtypestbl where d<=11.22BD;
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d<=11.22BD;
+select * from newtypestbl where d<=11.22BD;
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d<=12;
+select * from newtypestbl where d<=12;
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d<=12;
+select * from newtypestbl where d<=12;
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d in ('0.22', '1.0');
+select * from newtypestbl where d in ('0.22', '1.0');
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d in ('0.22', '1.0');
+select * from newtypestbl where d in ('0.22', '1.0');
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d in ('0.22', '11.22');
+select * from newtypestbl where d in ('0.22', '11.22');
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d in ('0.22', '11.22');
+select * from newtypestbl where d in ('0.22', '11.22');
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d in ('0.9', '1.0');
+select * from newtypestbl where d in ('0.9', '1.0');
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d in ('0.9', '1.0');
+select * from newtypestbl where d in ('0.9', '1.0');
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22);
+select * from newtypestbl where d in ('0.9', 0.22);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22);
+select * from newtypestbl where d in ('0.9', 0.22);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float));
+select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float));
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float));
+select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float));
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d between 0 and 1;
+select * from newtypestbl where d between 0 and 1;
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d between 0 and 1;
+select * from newtypestbl where d between 0 and 1;
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d between 0 and 1000;
+select * from newtypestbl where d between 0 and 1000;
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d between 0 and 1000;
+select * from newtypestbl where d between 0 and 1000;
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d between 0 and '2.0';
+select * from newtypestbl where d between 0 and '2.0';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d between 0 and '2.0';
+select * from newtypestbl where d between 0 and '2.0';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d between 0 and cast(3 as float);
+select * from newtypestbl where d between 0 and cast(3 as float);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d between 0 and cast(3 as float);
+select * from newtypestbl where d between 0 and cast(3 as float);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where d between 1 and cast(30 as char(10));
+select * from newtypestbl where d between 1 and cast(30 as char(10));
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where d between 1 and cast(30 as char(10));
+select * from newtypestbl where d between 1 and cast(30 as char(10));

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/queries/clientpositive/parquet_ppd_timestamp.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_ppd_timestamp.q b/ql/src/test/queries/clientpositive/parquet_ppd_timestamp.q
index e0802a0..6ed1e55 100644
--- a/ql/src/test/queries/clientpositive/parquet_ppd_timestamp.q
+++ b/ql/src/test/queries/clientpositive/parquet_ppd_timestamp.q
@@ -5,94 +5,94 @@ SET mapred.max.split.size=5000;
 
 create table newtypestbl(c char(10), v varchar(10), d decimal(5,3), ts timestamp) stored as parquet;
 
-insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("2011-01-01 01:01:01" as timestamp) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("2011-01-20 01:01:01" as timestamp) from src src2) uniontbl;
+insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("2011-01-01 01:01:01" as timestamp) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("2011-01-20 01:01:01" as timestamp) from src src2 limit 10) uniontbl;
 
 -- timestamp data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where cast(ts as string)='2011-01-01 01:01:01';
+select * from newtypestbl where cast(ts as string)='2011-01-01 01:01:01';
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where cast(ts as string)='2011-01-01 01:01:01';
+select * from newtypestbl where cast(ts as string)='2011-01-01 01:01:01';
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp);
+select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp);
+select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20));
+select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20));
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20));
+select * from newtypestbl where ts=cast('2011-01-01 01:01:01' as varchar(20));
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp);
+select * from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp);
+select * from newtypestbl where ts!=cast('2011-01-01 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp);
+select * from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp);
+select * from newtypestbl where ts<cast('2011-01-20 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp);
+select * from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp);
+select * from newtypestbl where ts<cast('2011-01-22 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp);
+select * from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp);
+select * from newtypestbl where ts<cast('2010-10-01 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp);
+select * from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp);
+select * from newtypestbl where ts<=cast('2011-01-01 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp);
+select * from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp);
+select * from newtypestbl where ts<=cast('2011-01-20 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp));
+select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp));
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp));
+select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp));
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp));
+select * from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp));
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp));
+select * from newtypestbl where ts in (cast('2011-01-01 01:01:01' as timestamp), cast('2011-01-20 01:01:01' as timestamp));
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp));
+select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp));
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp));
+select * from newtypestbl where ts in (cast('2011-01-02 01:01:01' as timestamp), cast('2011-01-08 01:01:01' as timestamp));
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp);
+select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp);
+select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-08 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp);
+select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp);
+select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2011-01-25 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp);
+select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp);
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp);
+select * from newtypestbl where ts between cast('2010-10-01 01:01:01' as timestamp) and cast('2010-11-01 01:01:01' as timestamp);

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/queries/clientpositive/parquet_ppd_varchar.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_ppd_varchar.q b/ql/src/test/queries/clientpositive/parquet_ppd_varchar.q
index be50ca2..41bf7df 100644
--- a/ql/src/test/queries/clientpositive/parquet_ppd_varchar.q
+++ b/ql/src/test/queries/clientpositive/parquet_ppd_varchar.q
@@ -5,72 +5,72 @@ SET mapred.max.split.size=5000;
 
 create table newtypestbl(c char(10), v varchar(10), d decimal(5,3), da date) stored as parquet;
 
-insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl;
+insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl;
 
 set hive.optimize.index.filter=false;
 
 -- varchar data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where v="bee";
+select * from newtypestbl where v="bee";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v="bee";
+select * from newtypestbl where v="bee";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where v!="bee";
+select * from newtypestbl where v!="bee";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v!="bee";
+select * from newtypestbl where v!="bee";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where v<"world";
+select * from newtypestbl where v<"world";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v<"world";
+select * from newtypestbl where v<"world";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where v<="world";
+select * from newtypestbl where v<="world";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v<="world";
+select * from newtypestbl where v<="world";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where v="bee   ";
+select * from newtypestbl where v="bee   ";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v="bee   ";
+select * from newtypestbl where v="bee   ";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where v in ("bee", "orange");
+select * from newtypestbl where v in ("bee", "orange");
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v in ("bee", "orange");
+select * from newtypestbl where v in ("bee", "orange");
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where v in ("bee", "world");
+select * from newtypestbl where v in ("bee", "world");
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v in ("bee", "world");
+select * from newtypestbl where v in ("bee", "world");
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where v in ("orange");
+select * from newtypestbl where v in ("orange");
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v in ("orange");
+select * from newtypestbl where v in ("orange");
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where v between "bee" and "orange";
+select * from newtypestbl where v between "bee" and "orange";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v between "bee" and "orange";
+select * from newtypestbl where v between "bee" and "orange";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where v between "bee" and "zombie";
+select * from newtypestbl where v between "bee" and "zombie";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v between "bee" and "zombie";
+select * from newtypestbl where v between "bee" and "zombie";
 
 set hive.optimize.index.filter=false;
-select sum(hash(*)) from newtypestbl where v between "orange" and "pine";
+select * from newtypestbl where v between "orange" and "pine";
 
 set hive.optimize.index.filter=true;
-select sum(hash(*)) from newtypestbl where v between "orange" and "pine";
\ No newline at end of file
+select * from newtypestbl where v between "orange" and "pine";
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/queries/clientpositive/parquet_predicate_pushdown.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_predicate_pushdown.q b/ql/src/test/queries/clientpositive/parquet_predicate_pushdown.q
index 32767e8..3f97dec 100644
--- a/ql/src/test/queries/clientpositive/parquet_predicate_pushdown.q
+++ b/ql/src/test/queries/clientpositive/parquet_predicate_pushdown.q
@@ -60,28 +60,32 @@ SET hive.optimize.index.filter=true;
 SELECT * FROM tbl_pred WHERE t>2 limit 1;
 SET hive.optimize.index.filter=false;
 
-SELECT SUM(HASH(t)) FROM tbl_pred
+SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
-  AND t > -2;
+  AND t > -2
+  LIMIT 10;
 
 SET hive.optimize.index.filter=true;
-SELECT SUM(HASH(t)) FROM tbl_pred
+SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
-  AND t > -2;
+  AND t > -2
+  LIMIT 10;
 SET hive.optimize.index.filter=false;
 
-EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+EXPLAIN SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
-  AND t > -2;
+  AND t > -2
+  LIMIT 10;
 
 SET hive.optimize.index.filter=true;
-EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+EXPLAIN SELECT * FROM tbl_pred
   WHERE t IS NOT NULL
   AND t < 0
-  AND t > -2;
+  AND t > -2
+  LIMIT 10;
 SET hive.optimize.index.filter=false;
 
 SELECT t, s FROM tbl_pred

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/results/clientpositive/parquet_ppd_boolean.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_ppd_boolean.q.out b/ql/src/test/results/clientpositive/parquet_ppd_boolean.q.out
index 78b7aa6..51ea879 100644
--- a/ql/src/test/results/clientpositive/parquet_ppd_boolean.q.out
+++ b/ql/src/test/results/clientpositive/parquet_ppd_boolean.q.out
@@ -6,11 +6,11 @@ POSTHOOK: query: create table newtypestbl(c char(10), v varchar(10), d decimal(5
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@newtypestbl
-PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, true from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, false from src src2) uniontbl
+PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, true from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, false from src src2 limit 10) uniontbl
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@newtypestbl
-POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, true from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, false from src src2) uniontbl
+POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, true from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, false from src src2 limit 10) uniontbl
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@newtypestbl
@@ -18,183 +18,253 @@ POSTHOOK: Lineage: newtypestbl.b EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.c EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.d EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.v EXPRESSION []
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b=true
+PREHOOK: query: select * from newtypestbl where b=true
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b=true
+POSTHOOK: query: select * from newtypestbl where b=true
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951953500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b!=true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+PREHOOK: query: select * from newtypestbl where b!=true
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b!=true
+POSTHOOK: query: select * from newtypestbl where b!=true
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427776000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b<true
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+PREHOOK: query: select * from newtypestbl where b<true
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b<true
+POSTHOOK: query: select * from newtypestbl where b<true
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427776000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b>true
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+PREHOOK: query: select * from newtypestbl where b>true
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b>true
+POSTHOOK: query: select * from newtypestbl where b>true
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b<=true
+PREHOOK: query: select * from newtypestbl where b<=true
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b<=true
+POSTHOOK: query: select * from newtypestbl where b<=true
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475822500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b=false
+hello     	world	11.22	false
+apple     	bee	0.22	true
+hello     	world	11.22	false
+apple     	bee	0.22	true
+hello     	world	11.22	false
+apple     	bee	0.22	true
+hello     	world	11.22	false
+apple     	bee	0.22	true
+hello     	world	11.22	false
+apple     	bee	0.22	true
+PREHOOK: query: select * from newtypestbl where b=false
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b=false
+POSTHOOK: query: select * from newtypestbl where b=false
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427776000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b!=false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+PREHOOK: query: select * from newtypestbl where b!=false
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b!=false
+POSTHOOK: query: select * from newtypestbl where b!=false
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951953500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b<false
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+PREHOOK: query: select * from newtypestbl where b<false
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b<false
+POSTHOOK: query: select * from newtypestbl where b<false
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b>false
+PREHOOK: query: select * from newtypestbl where b>false
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b>false
+POSTHOOK: query: select * from newtypestbl where b>false
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951953500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b<=false
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+PREHOOK: query: select * from newtypestbl where b<=false
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b<=false
+POSTHOOK: query: select * from newtypestbl where b<=false
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427776000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b=true
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+PREHOOK: query: select * from newtypestbl where b=true
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b=true
+POSTHOOK: query: select * from newtypestbl where b=true
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951953500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b!=true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+PREHOOK: query: select * from newtypestbl where b!=true
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b!=true
+POSTHOOK: query: select * from newtypestbl where b!=true
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427776000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b<true
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+PREHOOK: query: select * from newtypestbl where b<true
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b<true
+POSTHOOK: query: select * from newtypestbl where b<true
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427776000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b>true
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+PREHOOK: query: select * from newtypestbl where b>true
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b>true
+POSTHOOK: query: select * from newtypestbl where b>true
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b<=true
+PREHOOK: query: select * from newtypestbl where b<=true
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b<=true
+POSTHOOK: query: select * from newtypestbl where b<=true
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475822500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b=false
+hello     	world	11.22	false
+apple     	bee	0.22	true
+hello     	world	11.22	false
+apple     	bee	0.22	true
+hello     	world	11.22	false
+apple     	bee	0.22	true
+hello     	world	11.22	false
+apple     	bee	0.22	true
+hello     	world	11.22	false
+apple     	bee	0.22	true
+PREHOOK: query: select * from newtypestbl where b=false
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b=false
+POSTHOOK: query: select * from newtypestbl where b=false
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427776000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b!=false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+PREHOOK: query: select * from newtypestbl where b!=false
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b!=false
+POSTHOOK: query: select * from newtypestbl where b!=false
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951953500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b<false
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+PREHOOK: query: select * from newtypestbl where b<false
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b<false
+POSTHOOK: query: select * from newtypestbl where b<false
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b>false
+PREHOOK: query: select * from newtypestbl where b>false
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b>false
+POSTHOOK: query: select * from newtypestbl where b>false
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951953500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where b<=false
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+apple     	bee	0.22	true
+PREHOOK: query: select * from newtypestbl where b<=false
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where b<=false
+POSTHOOK: query: select * from newtypestbl where b<=false
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427776000
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false
+hello     	world	11.22	false

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/results/clientpositive/parquet_ppd_char.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_ppd_char.q.out b/ql/src/test/results/clientpositive/parquet_ppd_char.q.out
index e62462c..af4a13c 100644
--- a/ql/src/test/results/clientpositive/parquet_ppd_char.q.out
+++ b/ql/src/test/results/clientpositive/parquet_ppd_char.q.out
@@ -6,11 +6,11 @@ POSTHOOK: query: create table newtypestbl(c char(10), v varchar(10), d decimal(5
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@newtypestbl
-PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl
+PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@newtypestbl
-POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl
+POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@newtypestbl
@@ -19,202 +19,290 @@ POSTHOOK: Lineage: newtypestbl.d EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.da EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.v EXPRESSION []
 PREHOOK: query: -- char data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where c="apple"
+select * from newtypestbl where c="apple"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
 POSTHOOK: query: -- char data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where c="apple"
+select * from newtypestbl where c="apple"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c="apple"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c="apple"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c="apple"
+POSTHOOK: query: select * from newtypestbl where c="apple"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c!="apple"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c!="apple"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c!="apple"
+POSTHOOK: query: select * from newtypestbl where c!="apple"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c!="apple"
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where c!="apple"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c!="apple"
+POSTHOOK: query: select * from newtypestbl where c!="apple"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c<"hello"
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where c<"hello"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c<"hello"
+POSTHOOK: query: select * from newtypestbl where c<"hello"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c<"hello"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c<"hello"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c<"hello"
+POSTHOOK: query: select * from newtypestbl where c<"hello"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c<="hello"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c<="hello"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c<="hello"
+POSTHOOK: query: select * from newtypestbl where c<="hello"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c<="hello"
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c<="hello"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c<="hello"
+POSTHOOK: query: select * from newtypestbl where c<="hello"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c="apple "
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c="apple "
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c="apple "
+POSTHOOK: query: select * from newtypestbl where c="apple "
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c="apple "
+PREHOOK: query: select * from newtypestbl where c="apple "
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c="apple "
+POSTHOOK: query: select * from newtypestbl where c="apple "
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c in ("apple", "carrot")
+PREHOOK: query: select * from newtypestbl where c in ("apple", "carrot")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c in ("apple", "carrot")
+POSTHOOK: query: select * from newtypestbl where c in ("apple", "carrot")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c in ("apple", "carrot")
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c in ("apple", "carrot")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c in ("apple", "carrot")
+POSTHOOK: query: select * from newtypestbl where c in ("apple", "carrot")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c in ("apple", "hello")
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c in ("apple", "hello")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c in ("apple", "hello")
+POSTHOOK: query: select * from newtypestbl where c in ("apple", "hello")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c in ("apple", "hello")
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c in ("apple", "hello")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c in ("apple", "hello")
+POSTHOOK: query: select * from newtypestbl where c in ("apple", "hello")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c in ("carrot")
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c in ("carrot")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c in ("carrot")
+POSTHOOK: query: select * from newtypestbl where c in ("carrot")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c in ("carrot")
+PREHOOK: query: select * from newtypestbl where c in ("carrot")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c in ("carrot")
+POSTHOOK: query: select * from newtypestbl where c in ("carrot")
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c between "apple" and "carrot"
+PREHOOK: query: select * from newtypestbl where c between "apple" and "carrot"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c between "apple" and "carrot"
+POSTHOOK: query: select * from newtypestbl where c between "apple" and "carrot"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c between "apple" and "carrot"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c between "apple" and "carrot"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c between "apple" and "carrot"
+POSTHOOK: query: select * from newtypestbl where c between "apple" and "carrot"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c between "apple" and "zombie"
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c between "apple" and "zombie"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c between "apple" and "zombie"
+POSTHOOK: query: select * from newtypestbl where c between "apple" and "zombie"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c between "apple" and "zombie"
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c between "apple" and "zombie"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c between "apple" and "zombie"
+POSTHOOK: query: select * from newtypestbl where c between "apple" and "zombie"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c between "carrot" and "carrot1"
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where c between "carrot" and "carrot1"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c between "carrot" and "carrot1"
+POSTHOOK: query: select * from newtypestbl where c between "carrot" and "carrot1"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where c between "carrot" and "carrot1"
+PREHOOK: query: select * from newtypestbl where c between "carrot" and "carrot1"
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where c between "carrot" and "carrot1"
+POSTHOOK: query: select * from newtypestbl where c between "carrot" and "carrot1"
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL


[24/50] [abbrv] hive git commit: HIVE-11792: User explain in tez does not preserve ordering (Prasanth Jayachandran reviewed by Pengcheng Xiong)

Posted by xu...@apache.org.
HIVE-11792: User explain in tez does not preserve ordering (Prasanth Jayachandran reviewed by Pengcheng Xiong)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/da0be3db
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/da0be3db
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/da0be3db

Branch: refs/heads/spark
Commit: da0be3db7741e59813adcc197dc545b57dcc3c0a
Parents: 8bcd07d
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Mon Sep 14 02:57:19 2015 -0500
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Mon Sep 14 02:57:19 2015 -0500

----------------------------------------------------------------------
 .../hadoop/hive/common/jsonexplain/tez/Op.java  |    8 +-
 .../hive/common/jsonexplain/tez/Stage.java      |   14 +-
 .../common/jsonexplain/tez/TezJsonParser.java   |   17 +-
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |   24 +-
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |    4 +-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |    9 +-
 .../authorization_explain.q.java1.7.out         |    2 +-
 .../authorization_explain.q.java1.8.out         |    2 +-
 .../clientpositive/explain_dependency.q.out     |   18 +-
 .../clientpositive/explain_dependency2.q.out    |   16 +-
 .../results/clientpositive/input4.q.java1.7.out |    2 +-
 .../results/clientpositive/input4.q.java1.8.out |    2 +-
 .../results/clientpositive/join0.q.java1.7.out  |    2 +-
 .../results/clientpositive/join0.q.java1.8.out  |    4 +-
 .../results/clientpositive/parallel_join0.q.out |    2 +-
 .../clientpositive/plan_json.q.java1.7.out      |    2 +-
 .../clientpositive/plan_json.q.java1.8.out      |    2 +-
 .../clientpositive/tez/constprog_dpp.q.out      |    4 +-
 .../clientpositive/tez/explainuser_1.q.out      |  496 +--
 .../clientpositive/tez/explainuser_2.q.out      | 3280 +++++++++---------
 .../clientpositive/tez/explainuser_3.q.out      |   10 +-
 21 files changed, 1953 insertions(+), 1967 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Op.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Op.java b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Op.java
index 9ecba7c..d0c1037 100644
--- a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Op.java
+++ b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Op.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.common.jsonexplain.tez;
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -62,7 +62,7 @@ public final class Op {
       JSONObject mapjoinObj = opObject.getJSONObject("Map Join Operator");
       // get the map for posToVertex
       JSONObject verticeObj = mapjoinObj.getJSONObject("input vertices:");
-      Map<String, String> posToVertex = new HashMap<>();
+      Map<String, String> posToVertex = new LinkedHashMap<>();
       for (String pos : JSONObject.getNames(verticeObj)) {
         String vertexName = verticeObj.getString(pos);
         posToVertex.put(pos, vertexName);
@@ -83,7 +83,7 @@ public final class Op {
       // update the keys to use vertex name
       JSONObject keys = mapjoinObj.getJSONObject("keys:");
       if (keys.length() != 0) {
-        JSONObject newKeys = new JSONObject();
+        JSONObject newKeys = new JSONObject(new LinkedHashMap<>());
         for (String key : JSONObject.getNames(keys)) {
           String vertexName = posToVertex.get(key);
           if (vertexName != null) {
@@ -116,7 +116,7 @@ public final class Op {
   }
 
   /**
-   * @param out
+   * @param printer
    * @param indentFlag
    * @param branchOfJoinOp
    *          This parameter is used to show if it is a branch of a Join

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Stage.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Stage.java b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Stage.java
index c5a78b5..455d59f 100644
--- a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Stage.java
+++ b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Stage.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.common.jsonexplain.tez;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -28,8 +27,6 @@ import java.util.Map;
 import java.util.TreeMap;
 
 import org.apache.hadoop.fs.Path;
-import org.codehaus.jackson.JsonParseException;
-import org.codehaus.jackson.map.JsonMappingException;
 import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
@@ -176,16 +173,11 @@ public final class Stage {
    * @param opName
    * @param opObj
    * @return
-   * @throws JSONException
-   * @throws JsonParseException
-   * @throws JsonMappingException
-   * @throws IOException
    * @throws Exception
    *           This method address the create table operator, fetch operator,
    *           etc
    */
-  Op extractOp(String opName, JSONObject opObj) throws JSONException, JsonParseException,
-      JsonMappingException, IOException, Exception {
+  Op extractOp(String opName, JSONObject opObj) throws Exception {
     List<Attr> attrs = new ArrayList<>();
     Vertex v = null;
     if (opObj.length() > 0) {
@@ -198,7 +190,7 @@ public final class Stage {
           JSONObject attrObj = (JSONObject) o;
           if (attrObj.length() > 0) {
             if (name.equals("Processor Tree:")) {
-              JSONObject object = new JSONObject();
+              JSONObject object = new JSONObject(new LinkedHashMap<>());
               object.put(name, attrObj);
               v = new Vertex(null, object, parser);
               v.extractOpTree();
@@ -232,7 +224,7 @@ public final class Stage {
     return false;
   }
 
-  public void print(Printer printer, List<Boolean> indentFlag) throws JSONException, Exception {
+  public void print(Printer printer, List<Boolean> indentFlag) throws Exception {
     // print stagename
     if (parser.printSet.contains(this)) {
       printer.println(TezJsonParser.prefixString(indentFlag) + " Please refer to the previous "

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/TezJsonParser.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/TezJsonParser.java b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/TezJsonParser.java
index c6ee4f6..20ce27b 100644
--- a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/TezJsonParser.java
+++ b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/TezJsonParser.java
@@ -18,11 +18,10 @@
 
 package org.apache.hadoop.hive.common.jsonexplain.tez;
 
-import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -31,27 +30,23 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.common.jsonexplain.JsonParser;
-import org.codehaus.jackson.JsonParseException;
-import org.codehaus.jackson.map.JsonMappingException;
-import org.json.JSONException;
 import org.json.JSONObject;
 
 public final class TezJsonParser implements JsonParser {
-  public final Map<String, Stage> stages = new HashMap<String, Stage>();;
+  public final Map<String, Stage> stages = new LinkedHashMap<>();
   protected final Log LOG;
   // the object that has been printed.
-  public final Set<Object> printSet = new HashSet<>();
+  public final Set<Object> printSet = new LinkedHashSet<>();
   // the vertex that should be inlined. <Operator, list of Vertex that is
   // inlined>
-  public final Map<Op, List<Connection>> inlineMap = new HashMap<>();
+  public final Map<Op, List<Connection>> inlineMap = new LinkedHashMap<>();
 
   public TezJsonParser() {
     super();
     LOG = LogFactory.getLog(this.getClass().getName());
   }
 
-  public void extractStagesAndPlans(JSONObject inputObject) throws JSONException,
-      JsonParseException, JsonMappingException, Exception, IOException {
+  public void extractStagesAndPlans(JSONObject inputObject) throws Exception {
     // extract stages
     JSONObject dependency = inputObject.getJSONObject("STAGE DEPENDENCIES");
     if (dependency != null && dependency.length() > 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
index 21de3cf..a74a8ad 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
@@ -97,7 +97,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
       throws Exception {
     assert(work.getDependency());
 
-    JSONObject outJSONObject = new JSONObject();
+    JSONObject outJSONObject = new JSONObject(new LinkedHashMap<>());
     List<Map<String, String>> inputTableInfo = new ArrayList<Map<String, String>>();
     List<Map<String, String>> inputPartitionInfo = new ArrayList<Map<String, String>>();
     for (ReadEntity input: work.getInputs()) {
@@ -133,7 +133,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
   public JSONObject getJSONLogicalPlan(PrintStream out, ExplainWork work) throws Exception {
     isLogical = true;
 
-    JSONObject outJSONObject = new JSONObject();
+    JSONObject outJSONObject = new JSONObject(new LinkedHashMap<>());
     boolean jsonOutput = work.isFormatted();
     if (jsonOutput) {
       out = null;
@@ -181,7 +181,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
 
     // If the user asked for a formatted output, dump the json output
     // in the output stream
-    JSONObject outJSONObject = new JSONObject();
+    JSONObject outJSONObject = new JSONObject(new LinkedHashMap<>());
 
     if (jsonOutput) {
       out = null;
@@ -339,7 +339,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
     BaseSemanticAnalyzer analyzer = work.getAnalyzer();
     HiveOperation operation = SessionState.get().getHiveOperation();
 
-    JSONObject object = new JSONObject();
+    JSONObject object = new JSONObject(new LinkedHashMap<>());
     Object jsonInput = toJson("INPUTS", toString(analyzer.getInputs()), out, work);
     if (work.isFormatted()) {
       object.put("INPUTS", jsonInput);
@@ -402,7 +402,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
 
     TreeMap<Object, Object> tree = new TreeMap<Object, Object>();
     tree.putAll(mp);
-    JSONObject json = jsonOutput ? new JSONObject() : null;
+    JSONObject json = jsonOutput ? new JSONObject(new LinkedHashMap<>()) : null;
     if (out != null && hasHeader && !mp.isEmpty()) {
       out.println();
     }
@@ -446,7 +446,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
           }
           if (jsonOutput) {
             for (TezWork.Dependency dep: (List<TezWork.Dependency>)ent.getValue()) {
-              JSONObject jsonDep = new JSONObject();
+              JSONObject jsonDep = new JSONObject(new LinkedHashMap<>());
               jsonDep.put("parent", dep.getName());
               jsonDep.put("type", dep.getType());
               json.accumulate(ent.getKey().toString(), jsonDep);
@@ -475,7 +475,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
           }
           if (jsonOutput) {
             for (SparkWork.Dependency dep: (List<SparkWork.Dependency>) ent.getValue()) {
-              JSONObject jsonDep = new JSONObject();
+              JSONObject jsonDep = new JSONObject(new LinkedHashMap<>());
               jsonDep.put("parent", dep.getName());
               jsonDep.put("type", dep.getShuffleType());
               jsonDep.put("partitions", dep.getNumPartitions());
@@ -613,7 +613,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
       }
     }
 
-    JSONObject json = jsonOutput ? new JSONObject() : null;
+    JSONObject json = jsonOutput ? new JSONObject(new LinkedHashMap<>()) : null;
     // If this is an operator then we need to call the plan generation on the
     // conf and then the children
     if (work instanceof Operator) {
@@ -783,7 +783,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
 
     if (jsonOutput) {
       if (keyJSONObject != null) {
-        JSONObject ret = new JSONObject();
+        JSONObject ret = new JSONObject(new LinkedHashMap<>());
         ret.put(keyJSONObject, json);
         return ret;
       }
@@ -841,7 +841,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
       throws Exception {
 
     boolean first = true;
-    JSONObject json = jsonOutput ? new JSONObject() : null;
+    JSONObject json = jsonOutput ? new JSONObject(new LinkedHashMap<>()) : null;
     if (out != null) {
       out.print(indentString(indent));
       out.print(task.getId());
@@ -946,7 +946,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
       out.println("STAGE DEPENDENCIES:");
     }
 
-    JSONObject json = jsonOutput ? new JSONObject() : null;
+    JSONObject json = jsonOutput ? new JSONObject(new LinkedHashMap<>()) : null;
     for (Task task : tasks) {
       JSONObject jsonOut = outputDependencies(task, out, json, jsonOutput, appendTaskType, 2);
       if (jsonOutput && jsonOut != null) {
@@ -965,7 +965,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
       out.println("STAGE PLANS:");
     }
 
-    JSONObject json = jsonOutput ? new JSONObject() : null;
+    JSONObject json = jsonOutput ? new JSONObject(new LinkedHashMap<>()) : null;
     for (Task task : tasks) {
       outputPlan(task, out, json, isExtended, jsonOutput, 2);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
index 73263ee..3a6ec1a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -289,7 +290,8 @@ public class TezTask extends Task<TezWork> {
     DAG dag = DAG.create(work.getName());
 
     // set some info for the query
-    JSONObject json = new JSONObject().put("context", "Hive").put("description", ctx.getCmd());
+    JSONObject json = new JSONObject(new LinkedHashMap()).put("context", "Hive")
+        .put("description", ctx.getCmd());
     String dagInfo = json.toString();
 
     if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
index 87638da..2caa7ae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
@@ -17,8 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.hooks;
 
-import java.io.Serializable;
-import java.util.List;
+import java.util.LinkedHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
@@ -26,15 +25,11 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.exec.ExplainTask;
-import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
-import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.plan.ExplainWork;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
@@ -161,7 +156,7 @@ public class ATSHook implements ExecuteWithHookContext {
   TimelineEntity createPreHookEvent(String queryId, String query, JSONObject explainPlan,
       long startTime, String user, String requestuser, int numMrJobs, int numTezJobs, String opId) throws Exception {
 
-    JSONObject queryObj = new JSONObject();
+    JSONObject queryObj = new JSONObject(new LinkedHashMap<>());
     queryObj.put("queryText", query);
     queryObj.put("queryPlan", explainPlan);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/authorization_explain.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_explain.q.java1.7.out b/ql/src/test/results/clientpositive/authorization_explain.q.java1.7.out
index 8209c6a..b7ec209 100644
--- a/ql/src/test/results/clientpositive/authorization_explain.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/authorization_explain.q.java1.7.out
@@ -44,4 +44,4 @@ PREHOOK: query: explain formatted authorization use default
 PREHOOK: type: SWITCHDATABASE
 POSTHOOK: query: explain formatted authorization use default
 POSTHOOK: type: SWITCHDATABASE
-{"OUTPUTS":[],"INPUTS":["database:default"],"OPERATION":"SWITCHDATABASE","CURRENT_USER":"hive_test_user"}
+{"INPUTS":["database:default"],"OUTPUTS":[],"CURRENT_USER":"hive_test_user","OPERATION":"SWITCHDATABASE"}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/authorization_explain.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_explain.q.java1.8.out b/ql/src/test/results/clientpositive/authorization_explain.q.java1.8.out
index bb2bee9..b7ec209 100644
--- a/ql/src/test/results/clientpositive/authorization_explain.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/authorization_explain.q.java1.8.out
@@ -44,4 +44,4 @@ PREHOOK: query: explain formatted authorization use default
 PREHOOK: type: SWITCHDATABASE
 POSTHOOK: query: explain formatted authorization use default
 POSTHOOK: type: SWITCHDATABASE
-{"CURRENT_USER":"hive_test_user","OPERATION":"SWITCHDATABASE","INPUTS":["database:default"],"OUTPUTS":[]}
+{"INPUTS":["database:default"],"OUTPUTS":[],"CURRENT_USER":"hive_test_user","OPERATION":"SWITCHDATABASE"}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/explain_dependency.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_dependency.q.out b/ql/src/test/results/clientpositive/explain_dependency.q.out
index fa4dc59..dbfc482 100644
--- a/ql/src/test/results/clientpositive/explain_dependency.q.out
+++ b/ql/src/test/results/clientpositive/explain_dependency.q.out
@@ -70,14 +70,14 @@ POSTHOOK: query: -- Simple select queries, union queries and join queries
 EXPLAIN DEPENDENCY 
   SELECT key, count(1) FROM srcpart WHERE ds IS NOT NULL GROUP BY key
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: EXPLAIN DEPENDENCY 
   SELECT key, count(1) FROM (SELECT key, value FROM src) subq1 GROUP BY key
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY 
   SELECT key, count(1) FROM (SELECT key, value FROM src) subq1 GROUP BY key
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: EXPLAIN DEPENDENCY 
   SELECT * FROM (
     SELECT key, value FROM src UNION ALL SELECT key, value FROM srcpart WHERE ds IS NOT NULL
@@ -88,36 +88,36 @@ POSTHOOK: query: EXPLAIN DEPENDENCY
     SELECT key, value FROM src UNION ALL SELECT key, value FROM srcpart WHERE ds IS NOT NULL
   ) S1
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: EXPLAIN DEPENDENCY 
   SELECT S1.key, S2.value FROM src S1 JOIN srcpart S2 ON S1.key = S2.key WHERE ds IS NOT NULL
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY 
   SELECT S1.key, S2.value FROM src S1 JOIN srcpart S2 ON S1.key = S2.key WHERE ds IS NOT NULL
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: -- With views
 EXPLAIN DEPENDENCY SELECT * FROM V1
 PREHOOK: type: QUERY
 POSTHOOK: query: -- With views
 EXPLAIN DEPENDENCY SELECT * FROM V1
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@v1","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v1]"}]}
+{"input_tables":[{"tablename":"default@v1","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v1]"}],"input_partitions":[]}
 PREHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V2
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V2
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}]}
+{"input_tables":[{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V3
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V3
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@v3","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v3]"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v3]"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}]}
+{"input_tables":[{"tablename":"default@v3","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v3]"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v3]"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V4
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V4
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@v4","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@v1","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v4]"},{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v4]"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v4, default@v1]"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}]}
+{"input_tables":[{"tablename":"default@v4","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@v1","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v4]"},{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v4]"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v4, default@v1]"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: -- The table should show up in the explain dependency even if none
 -- of the partitions are selected.
 CREATE VIEW V5 as SELECT * FROM srcpart where ds = '10'
@@ -136,4 +136,4 @@ PREHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V5
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V5
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@v5","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v5]"}]}
+{"input_tables":[{"tablename":"default@v5","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v5]"}],"input_partitions":[]}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/explain_dependency2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_dependency2.q.out b/ql/src/test/results/clientpositive/explain_dependency2.q.out
index cb4e6c3..7973a60 100644
--- a/ql/src/test/results/clientpositive/explain_dependency2.q.out
+++ b/ql/src/test/results/clientpositive/explain_dependency2.q.out
@@ -8,14 +8,14 @@ POSTHOOK: query: -- This test is used for testing EXPLAIN DEPENDENCY command
 -- select from a table which does not involve a map-reduce job
 EXPLAIN DEPENDENCY SELECT * FROM src
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: -- select from a table which involves a map-reduce job
 EXPLAIN DEPENDENCY SELECT count(*) FROM src
 PREHOOK: type: QUERY
 POSTHOOK: query: -- select from a table which involves a map-reduce job
 EXPLAIN DEPENDENCY SELECT count(*) FROM src
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: -- select from a partitioned table which does not involve a map-reduce job
 -- and some partitions are being selected
 EXPLAIN DEPENDENCY SELECT * FROM srcpart where ds is not null
@@ -24,7 +24,7 @@ POSTHOOK: query: -- select from a partitioned table which does not involve a map
 -- and some partitions are being selected
 EXPLAIN DEPENDENCY SELECT * FROM srcpart where ds is not null
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: -- select from a partitioned table which does not involve a map-reduce job
 -- and none of the partitions are being selected
 EXPLAIN DEPENDENCY SELECT * FROM srcpart where ds = '1'
@@ -33,7 +33,7 @@ POSTHOOK: query: -- select from a partitioned table which does not involve a map
 -- and none of the partitions are being selected
 EXPLAIN DEPENDENCY SELECT * FROM srcpart where ds = '1'
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: -- select from a partitioned table which involves a map-reduce job
 -- and some partitions are being selected
 EXPLAIN DEPENDENCY SELECT count(*) FROM srcpart where ds is not null
@@ -42,7 +42,7 @@ POSTHOOK: query: -- select from a partitioned table which involves a map-reduce
 -- and some partitions are being selected
 EXPLAIN DEPENDENCY SELECT count(*) FROM srcpart where ds is not null
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: -- select from a partitioned table which involves a map-reduce job
 -- and none of the partitions are being selected
 EXPLAIN DEPENDENCY SELECT count(*) FROM srcpart where ds = '1'
@@ -51,7 +51,7 @@ POSTHOOK: query: -- select from a partitioned table which involves a map-reduce
 -- and none of the partitions are being selected
 EXPLAIN DEPENDENCY SELECT count(*) FROM srcpart where ds = '1'
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: create table tstsrcpart like srcpart
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
@@ -66,11 +66,11 @@ PREHOOK: type: QUERY
 POSTHOOK: query: -- select from a partitioned table with no partitions which does not involve a map-reduce job
 EXPLAIN DEPENDENCY SELECT * FROM tstsrcpart where ds is not null
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@tstsrcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@tstsrcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: -- select from a partitioned table with no partitions which involves a map-reduce job
 EXPLAIN DEPENDENCY SELECT count(*) FROM tstsrcpart where ds is not null
 PREHOOK: type: QUERY
 POSTHOOK: query: -- select from a partitioned table with no partitions which involves a map-reduce job
 EXPLAIN DEPENDENCY SELECT count(*) FROM tstsrcpart where ds is not null
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@tstsrcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@tstsrcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/input4.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/input4.q.java1.7.out b/ql/src/test/results/clientpositive/input4.q.java1.7.out
index dccf625..eaeedcb 100644
--- a/ql/src/test/results/clientpositive/input4.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/input4.q.java1.7.out
@@ -48,7 +48,7 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN FORMATTED
 SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias
 POSTHOOK: type: QUERY
-{"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"TableScan":{"alias:":"input4alias","children":{"Select Operator":{"expressions:":"value (type: string), key (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"ListSink":{}},"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE"}}}}},"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}}}
+{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"TableScan":{"alias:":"input4alias","Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"value (type: string), key (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"ListSink":{}}}}}}}}}}
 PREHOOK: query: SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias
 PREHOOK: type: QUERY
 PREHOOK: Input: default@input4

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/input4.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/input4.q.java1.8.out b/ql/src/test/results/clientpositive/input4.q.java1.8.out
index bc408de..eaeedcb 100644
--- a/ql/src/test/results/clientpositive/input4.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/input4.q.java1.8.out
@@ -48,7 +48,7 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN FORMATTED
 SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"Processor Tree:":{"TableScan":{"alias:":"input4alias","Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"ListSink":{}},"outputColumnNames:":["_col0","_col1"],"expressions:":"value (type: string), key (type: string)"}}}},"limit:":"-1"}}}}
+{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"TableScan":{"alias:":"input4alias","Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"value (type: string), key (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"ListSink":{}}}}}}}}}}
 PREHOOK: query: SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias
 PREHOOK: type: QUERY
 PREHOOK: Input: default@input4

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/join0.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join0.q.java1.7.out b/ql/src/test/results/clientpositive/join0.q.java1.7.out
index fa55ffb..f7e52f6 100644
--- a/ql/src/test/results/clientpositive/join0.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/join0.q.java1.7.out
@@ -118,7 +118,7 @@ SELECT src1.key as k1, src1.value as v1,
   (SELECT * FROM src WHERE src.key < 10) src2
   SORT BY k1, v1, k2, v2
 POSTHOOK: type: QUERY
-{"STAGE PLANS":{"Stage-2":{"Map Reduce":{"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"children":{"File Output Operator":{"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}}},"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE"}},"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"sort order:":"++++","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (ty
 pe: string), _col3 (type: string)"}}}}]}},"Stage-1":{"Map Reduce":{"Reduce Operator Tree:":{"Join Operator":{"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"children":{"File Output Operator":{"compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe","input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"}}},"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","condition map:":[{"":"Inner Join 0 to 1"}]}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 166 Data
  size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE"}},{"TableScan":{"alias:":"src","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats
 : COMPLETE Column stats: NONE"}}]}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}},"STAGE DEPENDENCIES":{"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)"}}}}}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COM
 PLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","outp
 ut format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"}}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"org.apache.hadoo
 p.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
 Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/join0.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join0.q.java1.8.out b/ql/src/test/results/clientpositive/join0.q.java1.8.out
index 443f6a3..f7e52f6 100644
--- a/ql/src/test/results/clientpositive/join0.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/join0.q.java1.8.out
@@ -1,5 +1,6 @@
 Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: -- JAVA_VERSION_SPECIFIC_OUTPUT
+-- SORT_QUERY_RESULTS
 
 EXPLAIN
 SELECT src1.key as k1, src1.value as v1, 
@@ -10,6 +11,7 @@ SELECT src1.key as k1, src1.value as v1,
   SORT BY k1, v1, k2, v2
 PREHOOK: type: QUERY
 POSTHOOK: query: -- JAVA_VERSION_SPECIFIC_OUTPUT
+-- SORT_QUERY_RESULTS
 
 EXPLAIN
 SELECT src1.key as k1, src1.value as v1, 
@@ -116,7 +118,7 @@ SELECT src1.key as k1, src1.value as v1,
   (SELECT * FROM src WHERE src.key < 10) src2
   SORT BY k1, v1, k2, v2
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Reduce Operator Tree:":{"Join Operator":{"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","keys:":{},"children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"}}},"condition map:":[{"":"Inner Join 0 to 1"}],"outputColumnNames:":["_col0","_col1","_col2","_col3"]}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"Statist
 ics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"value expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"outputColumnNames:":["_col0","_col1"],"expressions:":"key (type: string), value (type: string)"}},"predicate:":"(key < 10) (type: boolean)"}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"value expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"
 outputColumnNames:":["_col0","_col1"],"expressions:":"key (type: string), value (type: string)"}},"predicate:":"(key < 10) (type: boolean)"}}}}]}},"Stage-2":{"Map Reduce":{"Reduce Operator Tree:":{"Select Operator":{"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)"}},"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"sort order:":"++++","Statistics:":"Num rows: 182 
 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)"}}}}]}},"Stage-0":{"Fetch Operator":{"Processor Tree:":{"ListSink":{}},"limit:":"-1"}}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)"}}}}}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COM
 PLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","outp
 ut format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"}}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"org.apache.hadoo
 p.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
 Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/parallel_join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parallel_join0.q.out b/ql/src/test/results/clientpositive/parallel_join0.q.out
index 0f57a51..3aef936 100644
--- a/ql/src/test/results/clientpositive/parallel_join0.q.out
+++ b/ql/src/test/results/clientpositive/parallel_join0.q.out
@@ -116,7 +116,7 @@ SELECT src1.key as k1, src1.value as v1,
   (SELECT * FROM src WHERE src.key < 10) src2
   SORT BY k1, v1, k2, v2
 POSTHOOK: type: QUERY
-{"STAGE PLANS":{"Stage-2":{"Map Reduce":{"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"children":{"File Output Operator":{"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}}},"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE"}},"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"sort order:":"++++","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (ty
 pe: string), _col3 (type: string)"}}}}]}},"Stage-1":{"Map Reduce":{"Reduce Operator Tree:":{"Join Operator":{"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"children":{"File Output Operator":{"compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe","input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"}}},"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","condition map:":[{"":"Inner Join 0 to 1"}]}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 166 Data
  size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE"}},{"TableScan":{"alias:":"src","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats
 : COMPLETE Column stats: NONE"}}]}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}},"STAGE DEPENDENCIES":{"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)"}}}}}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COM
 PLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","outp
 ut format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"}}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"org.apache.hadoo
 p.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
 Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/plan_json.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/plan_json.q.java1.7.out b/ql/src/test/results/clientpositive/plan_json.q.java1.7.out
index 0979c70..2faa08a 100644
--- a/ql/src/test/results/clientpositive/plan_json.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/plan_json.q.java1.7.out
@@ -10,4 +10,4 @@ POSTHOOK: query: -- explain plan json:  the query gets the formatted json output
 
 EXPLAIN FORMATTED SELECT count(1) FROM src
 POSTHOOK: type: QUERY
-{"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Reduce Operator Tree:":{"Group By Operator":{"mode:":"mergepartial","aggregations:":["count(VALUE._col0)"],"outputColumnNames:":["_col0"],"children":{"File Output Operator":{"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}}},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE"}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","children":{"Select Operator":{"children":{"Group By Operator":{"mode:":"hash","aggregations:":["count(1)"],"outputColumnNames:":["_col0"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: bigint)","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
 "}},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE"}}]}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}},"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-1"}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-1"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Select Operator":{"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Group By Operator":{"aggregations:":["count(1)"],"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","value expressions:":"_col0 (type: bigint)"}}}}}}}}],"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column 
 stats: COMPLETE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","table:":{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/plan_json.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/plan_json.q.java1.8.out b/ql/src/test/results/clientpositive/plan_json.q.java1.8.out
index 13842b4..2faa08a 100644
--- a/ql/src/test/results/clientpositive/plan_json.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/plan_json.q.java1.8.out
@@ -10,4 +10,4 @@ POSTHOOK: query: -- explain plan json:  the query gets the formatted json output
 
 EXPLAIN FORMATTED SELECT count(1) FROM src
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-1"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"mode:":"mergepartial","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Select Operator":{"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","table:":{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}},"outputColumnNames:":["_col0"],"expressions:":"_col0 (type: bigint)"}},"outputColumnNames:":["_col0"]}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size
 : 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Select Operator":{"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Group By Operator":{"aggregations:":["count(_col0)"],"mode:":"hash","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Reduce Output Operator":{"value expressions:":"_col0 (type: bigint)","sort order:":"","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE"}},"outputColumnNames:":["_col0"]}},"outputColumnNames:":["_col0"],"expressions:":"1 (type: int)"}}}}]}},"Stage-0":{"Fetch Operator":{"Processor Tree:":{"ListSink":{}},"limit:":"-1"}}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-1"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Select Operator":{"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Group By Operator":{"aggregations:":["count(1)"],"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","value expressions:":"_col0 (type: bigint)"}}}}}}}}],"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column 
 stats: COMPLETE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","table:":{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/tez/constprog_dpp.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/constprog_dpp.q.out b/ql/src/test/results/clientpositive/tez/constprog_dpp.q.out
index b2b2371..0bc964b 100644
--- a/ql/src/test/results/clientpositive/tez/constprog_dpp.q.out
+++ b/ql/src/test/results/clientpositive/tez/constprog_dpp.q.out
@@ -54,10 +54,10 @@ Stage-0
          File Output Operator [FS_16]
             compressed:false
             Statistics:Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Merge Join Operator [MERGEJOIN_20]
             |  condition map:[{"":"Left Outer Join0 to 1"}]
-            |  keys:{"1":"_col0 (type: int)","0":"id (type: int)"}
+            |  keys:{"0":"id (type: int)","1":"_col0 (type: int)"}
             |  outputColumnNames:["_col0"]
             |  Statistics:Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             |<-Map 6 [SIMPLE_EDGE]


[16/50] [abbrv] hive git commit: HIVE-11614: CBO: Calcite Operator To Hive Operator (Calcite Return Path): ctas after order by has problem (Pengcheng Xiong, reviewd by Laljo John Pullokkaran)

Posted by xu...@apache.org.
HIVE-11614: CBO: Calcite Operator To Hive Operator (Calcite Return Path): ctas after order by has problem (Pengcheng Xiong, reviewd by Laljo John Pullokkaran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bbb91292
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bbb91292
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bbb91292

Branch: refs/heads/spark
Commit: bbb912927a1457daf283f3030cd873d55b93c8c3
Parents: ff1f5b1
Author: Pengcheng Xiong <px...@apache.org>
Authored: Sat Sep 12 20:27:16 2015 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Sat Sep 12 20:27:16 2015 -0700

----------------------------------------------------------------------
 .../translator/PlanModifierForReturnPath.java   |   4 -
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   7 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   2 +-
 .../queries/clientpositive/cbo_rp_auto_join17.q |  14 +
 .../cbo_rp_cross_product_check_2.q              |  31 +
 .../clientpositive/cbo_rp_auto_join17.q.out     | 118 ++++
 .../cbo_rp_cross_product_check_2.q.out          | 699 +++++++++++++++++++
 7 files changed, 866 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bbb91292/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
index 81cc474..95d692c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
@@ -34,10 +34,6 @@ public class PlanModifierForReturnPath {
 
     Pair<RelNode, RelNode> topSelparentPair = HiveCalciteUtil.getTopLevelSelect(newTopNode);
     PlanModifierUtil.fixTopOBSchema(newTopNode, topSelparentPair, resultSchema, false);
-    if (isCTAS) {
-      newTopNode = PlanModifierForASTConv.renameTopLevelSelectInResultSchema(newTopNode,
-          topSelparentPair, resultSchema);
-    }
 
     return newTopNode;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb91292/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 86bdf7e..8e992da 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -193,7 +193,6 @@ import com.google.common.collect.Lists;
 public class CalcitePlanner extends SemanticAnalyzer {
 
   private final AtomicInteger noColsMissingStats = new AtomicInteger(0);
-  private List<FieldSchema> topLevelFieldSchema;
   private SemanticException semanticException;
   private boolean           runCBO             = true;
 
@@ -620,7 +619,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       rethrowCalciteException(e);
       throw new AssertionError("rethrowCalciteException didn't throw for " + e.getMessage());
     }
-    optiqOptimizedAST = ASTConverter.convert(optimizedOptiqPlan, topLevelFieldSchema);
+    optiqOptimizedAST = ASTConverter.convert(optimizedOptiqPlan, resultSchema);
 
     return optiqOptimizedAST;
   }
@@ -644,7 +643,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
     }
 
     RelNode modifiedOptimizedOptiqPlan = PlanModifierForReturnPath.convertOpTree(
-        introduceProjectIfNeeded(optimizedOptiqPlan), topLevelFieldSchema, this.getQB()
+        introduceProjectIfNeeded(optimizedOptiqPlan), resultSchema, this.getQB()
             .getTableDesc() != null);
 
     LOG.debug("Translating the following plan:\n" + RelOptUtil.toString(modifiedOptimizedOptiqPlan));
@@ -851,7 +850,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       // 1. Gen Calcite Plan
       try {
         calciteGenPlan = genLogicalPlan(getQB(), true);
-        topLevelFieldSchema = SemanticAnalyzer.convertRowSchemaToResultSetSchema(
+        resultSchema = SemanticAnalyzer.convertRowSchemaToResultSetSchema(
             relToHiveRR.get(calciteGenPlan),
             HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_RESULTSET_USE_UNIQUE_COLUMN_NAMES));
       } catch (SemanticException e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb91292/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index f6052e3..16957b6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -261,7 +261,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   private final HashMap<String, SplitSample> nameToSplitSample;
   Map<GroupByOperator, Set<String>> groupOpToInputTables;
   Map<String, PrunedPartitionList> prunedPartitions;
-  private List<FieldSchema> resultSchema;
+  protected List<FieldSchema> resultSchema;
   private CreateViewDesc createVwDesc;
   private ArrayList<String> viewsExpanded;
   private ASTNode viewSelect;

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb91292/ql/src/test/queries/clientpositive/cbo_rp_auto_join17.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_rp_auto_join17.q b/ql/src/test/queries/clientpositive/cbo_rp_auto_join17.q
new file mode 100644
index 0000000..7e2f068
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/cbo_rp_auto_join17.q
@@ -0,0 +1,14 @@
+set hive.cbo.returnpath.hiveop=true;
+set hive.auto.convert.join = true;
+
+CREATE TABLE dest1(key1 INT, value1 STRING, key2 INT, value2 STRING) STORED AS TEXTFILE;
+
+explain
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.*;
+
+
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.*;
+
+SELECT sum(hash(dest1.key1,dest1.value1,dest1.key2,dest1.value2)) FROM dest1;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb91292/ql/src/test/queries/clientpositive/cbo_rp_cross_product_check_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_rp_cross_product_check_2.q b/ql/src/test/queries/clientpositive/cbo_rp_cross_product_check_2.q
new file mode 100644
index 0000000..6c35548
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/cbo_rp_cross_product_check_2.q
@@ -0,0 +1,31 @@
+set hive.cbo.returnpath.hiveop=true;
+set hive.explain.user=false;
+-- SORT_QUERY_RESULTS
+
+create table A as
+select * from src;
+
+create table B as
+select * from src order by key
+limit 10;
+
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.noconditionaltask=true;
+set hive.auto.convert.join.noconditionaltask.size=10000000;
+
+explain select * from A join B;
+
+explain select * from B d1 join B d2 on d1.key = d2.key join A;
+
+explain select * from A join 
+         (select d1.key 
+          from B d1 join B d2 on d1.key = d2.key 
+          where 1 = 1 group by d1.key) od1;
+
+explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1;
+
+explain select * from 
+(select A.key from A group by key) ss join 
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1;
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb91292/ql/src/test/results/clientpositive/cbo_rp_auto_join17.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join17.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join17.q.out
new file mode 100644
index 0000000..351699d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join17.q.out
@@ -0,0 +1,118 @@
+PREHOOK: query: CREATE TABLE dest1(key1 INT, value1 STRING, key2 INT, value2 STRING) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@dest1
+POSTHOOK: query: CREATE TABLE dest1(key1 INT, value1 STRING, key2 INT, value2 STRING) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@dest1
+PREHOOK: query: explain
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.*
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.*
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-5 is a root stage
+  Stage-4 depends on stages: Stage-5
+  Stage-0 depends on stages: Stage-4
+  Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-5
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        src1 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        src1 
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: key, value
+                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 key (type: string)
+                    1 key (type: string)
+
+  Stage: Stage-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src2
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: key, value
+                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 key (type: string)
+                    1 key (type: string)
+                  outputColumnNames: key, value, key0, value0
+                  Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: UDFToInteger(key) (type: int), value (type: string), UDFToInteger(key0) (type: int), value0 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.TextInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                          name: default.dest1
+      Local Work:
+        Map Reduce Local Work
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: true
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.dest1
+
+  Stage: Stage-2
+    Stats-Aggr Operator
+
+PREHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.*
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@dest1
+POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.*
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@dest1
+POSTHOOK: Lineage: dest1.key1 EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: dest1.key2 EXPRESSION [(src)src2.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: dest1.value1 SIMPLE [(src)src1.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest1.value2 SIMPLE [(src)src2.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: SELECT sum(hash(dest1.key1,dest1.value1,dest1.key2,dest1.value2)) FROM dest1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dest1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(dest1.key1,dest1.value1,dest1.key2,dest1.value2)) FROM dest1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dest1
+#### A masked pattern was here ####
+-793937029770

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb91292/ql/src/test/results/clientpositive/cbo_rp_cross_product_check_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_cross_product_check_2.q.out b/ql/src/test/results/clientpositive/cbo_rp_cross_product_check_2.q.out
new file mode 100644
index 0000000..cdd47b6
--- /dev/null
+++ b/ql/src/test/results/clientpositive/cbo_rp_cross_product_check_2.q.out
@@ -0,0 +1,699 @@
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+create table A as
+select * from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@A
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+create table A as
+select * from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@A
+PREHOOK: query: create table B as
+select * from src order by key
+limit 10
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@B
+POSTHOOK: query: create table B as
+select * from src order by key
+limit 10
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@B
+Warning: Map Join MAPJOIN[9][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
+PREHOOK: query: explain select * from A join B
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join B
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-4 is a root stage
+  Stage-3 depends on stages: Stage-4
+  Stage-0 depends on stages: Stage-3
+
+STAGE PLANS:
+  Stage: Stage-4
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        b 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        b 
+          TableScan
+            alias: b
+            Statistics: Num rows: 10 Data size: 96 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: key, value
+              Statistics: Num rows: 10 Data size: 96 Basic stats: COMPLETE Column stats: NONE
+              HashTable Sink Operator
+                keys:
+                  0 
+                  1 
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: key, value
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: key, value, key0, value0
+                Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: key (type: string), value (type: string), key0 (type: string), value0 (type: string)
+                  outputColumnNames: key, value, key0, value0
+                  Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Local Work:
+        Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Map Join MAPJOIN[17][bigTable=?] in task 'Stage-5:MAPRED' is a cross product
+PREHOOK: query: explain select * from B d1 join B d2 on d1.key = d2.key join A
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from B d1 join B d2 on d1.key = d2.key join A
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-7 is a root stage
+  Stage-5 depends on stages: Stage-7
+  Stage-0 depends on stages: Stage-5
+
+STAGE PLANS:
+  Stage: Stage-7
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        a 
+          Fetch Operator
+            limit: -1
+        d1 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        a 
+          TableScan
+            alias: a
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            HashTable Sink Operator
+              keys:
+                0 
+                1 
+        d1 
+          TableScan
+            alias: d1
+            Statistics: Num rows: 10 Data size: 96 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 5 Data size: 48 Basic stats: COMPLETE Column stats: NONE
+              HashTable Sink Operator
+                keys:
+                  0 key (type: string)
+                  1 key (type: string)
+
+  Stage: Stage-5
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: d2
+            Statistics: Num rows: 10 Data size: 96 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 5 Data size: 48 Basic stats: COMPLETE Column stats: NONE
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 key (type: string)
+                  1 key (type: string)
+                outputColumnNames: _col0, _col1, _col5, _col6
+                Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 
+                    1 
+                  outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
+                  Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                    Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.TextInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Local Work:
+        Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Map Join MAPJOIN[25][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+PREHOOK: query: explain select * from A join 
+         (select d1.key 
+          from B d1 join B d2 on d1.key = d2.key 
+          where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join 
+         (select d1.key 
+          from B d1 join B d2 on d1.key = d2.key 
+          where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-9 is a root stage
+  Stage-2 depends on stages: Stage-9
+  Stage-8 depends on stages: Stage-2
+  Stage-6 depends on stages: Stage-8
+  Stage-0 depends on stages: Stage-6
+
+STAGE PLANS:
+  Stage: Stage-9
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        od1:d1 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        od1:d1 
+          TableScan
+            alias: d1
+            Statistics: Num rows: 10 Data size: 96 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 5 Data size: 48 Basic stats: COMPLETE Column stats: NONE
+              HashTable Sink Operator
+                keys:
+                  0 key (type: string)
+                  1 key (type: string)
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: d2
+            Statistics: Num rows: 10 Data size: 96 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 5 Data size: 48 Basic stats: COMPLETE Column stats: NONE
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 key (type: string)
+                  1 key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: string)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    keys: _col0 (type: string)
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE
+      Local Work:
+        Map Reduce Local Work
+      Reduce Operator Tree:
+        Group By Operator
+          keys: KEY._col0 (type: string)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: string)
+            outputColumnNames: _col0
+            Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-8
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        a 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        a 
+          TableScan
+            alias: a
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            HashTable Sink Operator
+              keys:
+                0 
+                1 
+
+  Stage: Stage-6
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Map Join Operator
+              condition map:
+                   Inner Join 0 to 1
+              keys:
+                0 
+                1 
+              outputColumnNames: _col0, _col1, _col5
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Local Work:
+        Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[22][bigTable=d2] in task 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-9 is a root stage
+  Stage-2 depends on stages: Stage-9
+  Stage-8 depends on stages: Stage-2
+  Stage-6 depends on stages: Stage-8
+  Stage-0 depends on stages: Stage-6
+
+STAGE PLANS:
+  Stage: Stage-9
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        od1:d1 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        od1:d1 
+          TableScan
+            alias: d1
+            Statistics: Num rows: 10 Data size: 96 Basic stats: COMPLETE Column stats: NONE
+            HashTable Sink Operator
+              keys:
+                0 
+                1 
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: d2
+            Statistics: Num rows: 10 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+            Map Join Operator
+              condition map:
+                   Inner Join 0 to 1
+              keys:
+                0 
+                1 
+              outputColumnNames: _col0
+              Statistics: Num rows: 11 Data size: 105 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: _col0 (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 11 Data size: 105 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  keys: _col0 (type: string)
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 11 Data size: 105 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: string)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: string)
+                    Statistics: Num rows: 11 Data size: 105 Basic stats: COMPLETE Column stats: NONE
+      Local Work:
+        Map Reduce Local Work
+      Reduce Operator Tree:
+        Group By Operator
+          keys: KEY._col0 (type: string)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 5 Data size: 47 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: string)
+            outputColumnNames: _col0
+            Statistics: Num rows: 5 Data size: 47 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-8
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        a 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        a 
+          TableScan
+            alias: a
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            HashTable Sink Operator
+              keys:
+                0 
+                1 
+
+  Stage: Stage-6
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Map Join Operator
+              condition map:
+                   Inner Join 0 to 1
+              keys:
+                0 
+                1 
+              outputColumnNames: _col0, _col1, _col5
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Local Work:
+        Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Map Join MAPJOIN[47][bigTable=?] in task 'Stage-7:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[39][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[21][tables = [ss, od1]] in Stage 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: explain select * from 
+(select A.key from A group by key) ss join 
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from 
+(select A.key from A group by key) ss join 
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-8 depends on stages: Stage-1, Stage-4 , consists of Stage-10, Stage-11, Stage-2
+  Stage-10 has a backup stage: Stage-2
+  Stage-6 depends on stages: Stage-10
+  Stage-11 has a backup stage: Stage-2
+  Stage-7 depends on stages: Stage-11
+  Stage-2
+  Stage-12 is a root stage
+  Stage-4 depends on stages: Stage-12
+  Stage-0 depends on stages: Stage-6, Stage-7, Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string)
+              outputColumnNames: key
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                keys: key (type: string)
+                mode: hash
+                outputColumnNames: _col0
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Group By Operator
+          keys: KEY._col0 (type: string)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: string)
+            outputColumnNames: _col0
+            Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-8
+    Conditional Operator
+
+  Stage: Stage-10
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        $INTNAME1 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        $INTNAME1 
+          TableScan
+            HashTable Sink Operator
+              keys:
+                0 
+                1 
+
+  Stage: Stage-6
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Map Join Operator
+              condition map:
+                   Inner Join 0 to 1
+              keys:
+                0 
+                1 
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: _col0 (type: string), _col1 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Local Work:
+        Map Reduce Local Work
+
+  Stage: Stage-11
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        $INTNAME 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        $INTNAME 
+          TableScan
+            HashTable Sink Operator
+              keys:
+                0 
+                1 
+
+  Stage: Stage-7
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Map Join Operator
+              condition map:
+                   Inner Join 0 to 1
+              keys:
+                0 
+                1 
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: _col0 (type: string), _col1 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Local Work:
+        Map Reduce Local Work
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string)
+          TableScan
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: string), _col1 (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-12
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        od1:d1 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        od1:d1 
+          TableScan
+            alias: d1
+            Statistics: Num rows: 10 Data size: 96 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 5 Data size: 48 Basic stats: COMPLETE Column stats: NONE
+              HashTable Sink Operator
+                keys:
+                  0 key (type: string)
+                  1 key (type: string)
+
+  Stage: Stage-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: d2
+            Statistics: Num rows: 10 Data size: 96 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 5 Data size: 48 Basic stats: COMPLETE Column stats: NONE
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 key (type: string)
+                  1 key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: string)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    keys: _col0 (type: string)
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE
+      Local Work:
+        Map Reduce Local Work
+      Reduce Operator Tree:
+        Group By Operator
+          keys: KEY._col0 (type: string)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: string)
+            outputColumnNames: _col0
+            Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+


[45/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/auto_join18.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_join18.q.out b/ql/src/test/results/clientpositive/spark/auto_join18.q.out
index 8607221..349e76d 100644
--- a/ql/src/test/results/clientpositive/spark/auto_join18.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_join18.q.out
@@ -45,11 +45,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: count(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -66,10 +66,10 @@ STAGE PLANS:
                   Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string), _col1 (type: string)
+                      keys: key (type: string), value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out b/ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out
index df737c2..92c827e 100644
--- a/ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_join18_multi_distinct.q.out
@@ -47,11 +47,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: count(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -68,11 +68,11 @@ STAGE PLANS:
                   Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1), count(DISTINCT _col0)
-                      keys: _col0 (type: string), _col1 (type: string)
+                      aggregations: count(DISTINCT value), count(DISTINCT key)
+                      keys: key (type: string), value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/auto_join27.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_join27.q.out b/ql/src/test/results/clientpositive/spark/auto_join27.q.out
index 43313e0..a325533 100644
--- a/ql/src/test/results/clientpositive/spark/auto_join27.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_join27.q.out
@@ -60,20 +60,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (UDFToDouble(key) < 200.0) (type: boolean)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      keys: key (type: string), value (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: string), _col1 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: string)
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                          Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
         Map 6 
             Map Operator Tree:
                 TableScan

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/auto_join32.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_join32.q.out b/ql/src/test/results/clientpositive/spark/auto_join32.q.out
index 316792b..679dd79 100644
--- a/ql/src/test/results/clientpositive/spark/auto_join32.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_join32.q.out
@@ -422,7 +422,8 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2)
+        Reducer 2 <- Map 1 (GROUP, 2)
+        Reducer 3 <- Reducer 2 (GROUP, 2)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -445,20 +446,16 @@ STAGE PLANS:
                           1 _col0 (type: string)
                         outputColumnNames: _col1, _col3
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                        Select Operator
-                          expressions: _col3 (type: string), _col1 (type: string)
+                        Group By Operator
+                          keys: _col1 (type: string), _col3 (type: string)
+                          mode: hash
                           outputColumnNames: _col0, _col1
                           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                          Group By Operator
-                            keys: _col0 (type: string), _col1 (type: string)
-                            mode: hash
-                            outputColumnNames: _col0, _col1
+                          Reduce Output Operator
+                            key expressions: _col0 (type: string), _col1 (type: string)
+                            sort order: ++
+                            Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                            Reduce Output Operator
-                              key expressions: _col0 (type: string), _col1 (type: string)
-                              sort order: ++
-                              Map-reduce partition columns: _col0 (type: string)
-                              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
@@ -467,18 +464,32 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
-                  aggregations: count(_col1)
-                  keys: _col0 (type: string)
-                  mode: complete
+                  aggregations: count(_col0)
+                  keys: _col1 (type: string)
+                  mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                  File Output Operator
-                    compressed: false
+                  Reduce Output Operator
+                    key expressions: _col0 (type: string)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.TextInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    value expressions: _col1 (type: bigint)
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/count.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/count.q.out b/ql/src/test/results/clientpositive/spark/count.q.out
index b2e9ffb..cb9eda5 100644
--- a/ql/src/test/results/clientpositive/spark/count.q.out
+++ b/ql/src/test/results/clientpositive/spark/count.q.out
@@ -53,11 +53,11 @@ STAGE PLANS:
                   Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: a (type: int), b (type: int), c (type: int), d (type: int)
-                    outputColumnNames: _col0, _col1, _col2, _col3
+                    outputColumnNames: a, b, c, d
                     Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1), count(DISTINCT _col2), sum(_col3)
-                      keys: _col0 (type: int), _col1 (type: int), _col2 (type: int)
+                      aggregations: count(DISTINCT b), count(DISTINCT c), sum(d)
+                      keys: a (type: int), b (type: int), c (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                       Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
@@ -188,14 +188,14 @@ STAGE PLANS:
                   Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: a (type: int), b (type: int), c (type: int), d (type: int)
-                    outputColumnNames: _col0, _col1, _col2, _col3
+                    outputColumnNames: a, b, c, d
                     Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int)
+                      key expressions: a (type: int), b (type: int), c (type: int)
                       sort order: +++
-                      Map-reduce partition columns: _col0 (type: int)
+                      Map-reduce partition columns: a (type: int)
                       Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col3 (type: int)
+                      value expressions: d (type: int)
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/groupby5_map.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby5_map.q.out b/ql/src/test/results/clientpositive/spark/groupby5_map.q.out
index e775921..add3094 100644
--- a/ql/src/test/results/clientpositive/spark/groupby5_map.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby5_map.q.out
@@ -31,10 +31,10 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(key)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/groupby5_map_skew.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby5_map_skew.q.out b/ql/src/test/results/clientpositive/spark/groupby5_map_skew.q.out
index 5c58934..924ef5d 100644
--- a/ql/src/test/results/clientpositive/spark/groupby5_map_skew.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby5_map_skew.q.out
@@ -31,10 +31,10 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col0)
+                      aggregations: sum(key)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/groupby_cube1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_cube1.q.out b/ql/src/test/results/clientpositive/spark/groupby_cube1.q.out
index 25bfbec..8b5a4c0 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_cube1.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_cube1.q.out
@@ -223,11 +223,11 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), val (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, val
                     Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1)
-                      keys: _col0 (type: string), '0' (type: string), _col1 (type: string)
+                      aggregations: count(DISTINCT val)
+                      keys: key (type: string), '0' (type: string), val (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 2 Data size: 60 Basic stats: COMPLETE Column stats: NONE
@@ -408,11 +408,11 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), val (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, val
                     Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1)
-                      keys: _col0 (type: string), '0' (type: string), _col1 (type: string)
+                      aggregations: count(DISTINCT val)
+                      keys: key (type: string), '0' (type: string), val (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 2 Data size: 60 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/groupby_position.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_position.q.out b/ql/src/test/results/clientpositive/spark/groupby_position.q.out
index 088428b..99223b9 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_position.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_position.q.out
@@ -588,20 +588,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (((((UDFToDouble(key) > 15.0) and (UDFToDouble(key) < 25.0)) and (UDFToDouble(key) > 10.0)) and (UDFToDouble(key) < 20.0)) and key is not null) (type: boolean)
                     Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      keys: key (type: string), value (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: string), _col1 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: string)
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                          Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/groupby_resolution.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_resolution.q.out b/ql/src/test/results/clientpositive/spark/groupby_resolution.q.out
index 9d36099..cb2c9bd 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_resolution.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_resolution.q.out
@@ -20,12 +20,12 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: string)
+                      key expressions: key (type: string)
                       sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                      Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
@@ -71,12 +71,12 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: string)
+                      key expressions: key (type: string)
                       sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                      Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
@@ -123,10 +123,10 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: string)
+                      key expressions: key (type: string)
                       sort order: +
                       Map-reduce partition columns: rand() (type: double)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -189,10 +189,10 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: string)
+                      key expressions: key (type: string)
                       sort order: +
                       Map-reduce partition columns: rand() (type: double)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -254,11 +254,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: string)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -312,11 +312,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: string)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -371,11 +371,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: string)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -444,11 +444,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: string)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -620,22 +620,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (key < '12') (type: boolean)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string)
-                      outputColumnNames: _col0
+                    Group By Operator
+                      aggregations: count()
+                      keys: key (type: string)
+                      mode: hash
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: count()
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: rand() (type: double)
                         Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: rand() (type: double)
-                          Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: bigint)
+                        value expressions: _col1 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out b/ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out
index 0a06c65..ce5998e 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out
@@ -127,11 +127,11 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), val (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, val
                     Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1)
-                      keys: _col0 (type: string), '0' (type: string), _col1 (type: string)
+                      aggregations: count(DISTINCT val)
+                      keys: key (type: string), '0' (type: string), val (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 2 Data size: 60 Basic stats: COMPLETE Column stats: NONE
@@ -306,11 +306,11 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), val (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, val
                     Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1)
-                      keys: _col0 (type: string), '0' (type: string), _col1 (type: string)
+                      aggregations: count(DISTINCT val)
+                      keys: key (type: string), '0' (type: string), val (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 2 Data size: 60 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/having.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/having.q.out b/ql/src/test/results/clientpositive/spark/having.q.out
index da59af9..ad3ef79 100644
--- a/ql/src/test/results/clientpositive/spark/having.q.out
+++ b/ql/src/test/results/clientpositive/spark/having.q.out
@@ -110,22 +110,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (UDFToDouble(key) <> 302.0) (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      aggregations: max(value)
+                      keys: key (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: max(_col1)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: string)
+                        value expressions: _col1 (type: string)
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
@@ -486,11 +482,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: max(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -759,22 +755,18 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (UDFToDouble(key) > 300.0) (type: boolean)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
+                    Group By Operator
+                      aggregations: max(value)
+                      keys: key (type: string)
+                      mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: max(_col1)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: string)
+                        value expressions: _col1 (type: string)
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
@@ -959,11 +951,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: max(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1227,11 +1219,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: count(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/infer_bucket_sort_map_operators.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_map_operators.q.out b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_map_operators.q.out
index f39cd57..9b284e7 100644
--- a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_map_operators.q.out
+++ b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_map_operators.q.out
@@ -79,11 +79,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      keys: _col0 (type: string)
+                      keys: key (type: string)
                       mode: final
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/join18.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join18.q.out b/ql/src/test/results/clientpositive/spark/join18.q.out
index c702fdf..e11ecec 100644
--- a/ql/src/test/results/clientpositive/spark/join18.q.out
+++ b/ql/src/test/results/clientpositive/spark/join18.q.out
@@ -48,11 +48,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: count(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -69,10 +69,10 @@ STAGE PLANS:
                   Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string), _col1 (type: string)
+                      keys: key (type: string), value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out b/ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out
index 18d79d3..5560226 100644
--- a/ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/spark/join18_multi_distinct.q.out
@@ -50,11 +50,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: count(value)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -71,11 +71,11 @@ STAGE PLANS:
                   Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: key, value
                     Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1), count(DISTINCT _col0)
-                      keys: _col0 (type: string), _col1 (type: string)
+                      aggregations: count(DISTINCT value), count(DISTINCT key)
+                      keys: key (type: string), value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/join31.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join31.q.out b/ql/src/test/results/clientpositive/spark/join31.q.out
index 108a1ea..469a19f 100644
--- a/ql/src/test/results/clientpositive/spark/join31.q.out
+++ b/ql/src/test/results/clientpositive/spark/join31.q.out
@@ -49,20 +49,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string)
+                    Group By Operator
+                      keys: key (type: string)
+                      mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
         Reducer 5 
             Local Work:
               Map Reduce Local Work
@@ -92,20 +88,16 @@ STAGE PLANS:
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string)
+                    Group By Operator
+                      keys: key (type: string)
+                      mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Local Work:
               Map Reduce Local Work

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out b/ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out
index a34a399..6ca0527 100644
--- a/ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out
+++ b/ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out
@@ -553,10 +553,10 @@ STAGE PLANS:
                   Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: hr (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: hr
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: string)
+                      keys: hr (type: string)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/limit_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/limit_pushdown.q.out b/ql/src/test/results/clientpositive/spark/limit_pushdown.q.out
index 2dc710c..b0e6c6a 100644
--- a/ql/src/test/results/clientpositive/spark/limit_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/spark/limit_pushdown.q.out
@@ -384,10 +384,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: cdouble (type: double)
-                    outputColumnNames: _col0
+                    outputColumnNames: cdouble
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: double)
+                      keys: cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
@@ -473,10 +473,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint), cdouble (type: double)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: ctinyint, cdouble
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: tinyint), _col1 (type: double)
+                      keys: ctinyint (type: tinyint), cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
@@ -567,10 +567,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint), cdouble (type: double)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: ctinyint, cdouble
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: tinyint), _col1 (type: double)
+                      keys: ctinyint (type: tinyint), cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
@@ -663,11 +663,11 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ctinyint (type: tinyint), cstring1 (type: string), cstring2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
+                    outputColumnNames: ctinyint, cstring1, cstring2
                     Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(DISTINCT _col1), count(DISTINCT _col2)
-                      keys: _col0 (type: tinyint), _col1 (type: string), _col2 (type: string)
+                      aggregations: count(DISTINCT cstring1), count(DISTINCT cstring2)
+                      keys: ctinyint (type: tinyint), cstring1 (type: string), cstring2 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE
@@ -783,11 +783,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: value (type: string), key (type: string)
-                    outputColumnNames: _col0, _col1
+                    outputColumnNames: value, key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col1)
-                      keys: _col0 (type: string)
+                      aggregations: sum(key)
+                      keys: value (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1042,16 +1042,16 @@ STAGE PLANS:
                   alias: src
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: value (type: string), key (type: string)
-                    outputColumnNames: _col0, _col1
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: string)
+                      key expressions: value (type: string)
                       sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                      Map-reduce partition columns: value (type: string)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       TopN Hash Memory Usage: 0.3
-                      value expressions: _col1 (type: string)
+                      value expressions: key (type: string)
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out b/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
index 3b2c80e..549911c 100644
--- a/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
@@ -664,22 +664,18 @@ STAGE PLANS:
                           input vertices:
                             1 Map 4
                           Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                          Select Operator
-                            expressions: _col2 (type: string)
-                            outputColumnNames: _col0
+                          Group By Operator
+                            aggregations: count()
+                            keys: _col2 (type: string)
+                            mode: hash
+                            outputColumnNames: _col0, _col1
                             Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                            Group By Operator
-                              aggregations: count()
-                              keys: _col0 (type: string)
-                              mode: hash
-                              outputColumnNames: _col0, _col1
+                            Reduce Output Operator
+                              key expressions: _col0 (type: string)
+                              sort order: +
+                              Map-reduce partition columns: _col0 (type: string)
                               Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                              Reduce Output Operator
-                                key expressions: _col0 (type: string)
-                                sort order: +
-                                Map-reduce partition columns: _col0 (type: string)
-                                Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-                                value expressions: _col1 (type: bigint)
+                              value expressions: _col1 (type: bigint)
             Local Work:
               Map Reduce Local Work
         Reducer 2 

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/metadata_only_queries.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/metadata_only_queries.q.out b/ql/src/test/results/clientpositive/spark/metadata_only_queries.q.out
index 18e1d4e..b2221fc 100644
--- a/ql/src/test/results/clientpositive/spark/metadata_only_queries.q.out
+++ b/ql/src/test/results/clientpositive/spark/metadata_only_queries.q.out
@@ -448,10 +448,10 @@ STAGE PLANS:
                   Statistics: Num rows: 9489 Data size: 978785 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: ts (type: timestamp)
-                    outputColumnNames: _col0
+                    outputColumnNames: ts
                     Statistics: Num rows: 9489 Data size: 978785 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(_col0)
+                      aggregations: count(ts)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out b/ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out
index ed536c2..96defa6 100644
--- a/ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out
+++ b/ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out
@@ -215,24 +215,20 @@ STAGE PLANS:
                     isSamplingPred: false
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string)
-                      outputColumnNames: _col0
+                    Group By Operator
+                      aggregations: min(key)
+                      keys: key (type: string)
+                      mode: hash
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: min(_col0)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                          tag: -1
-                          value expressions: _col1 (type: string)
-                          auto parallelism: false
+                        tag: -1
+                        value expressions: _col1 (type: string)
+                        auto parallelism: false
             Path -> Alias:
 #### A masked pattern was here ####
             Path -> Partition:
@@ -601,24 +597,20 @@ STAGE PLANS:
                     isSamplingPred: false
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string)
-                      outputColumnNames: _col0
+                    Group By Operator
+                      aggregations: min(key)
+                      keys: key (type: string)
+                      mode: hash
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: min(_col0)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                          tag: -1
-                          value expressions: _col1 (type: string)
-                          auto parallelism: false
+                        tag: -1
+                        value expressions: _col1 (type: string)
+                        auto parallelism: false
             Path -> Alias:
 #### A masked pattern was here ####
             Path -> Partition:
@@ -981,11 +973,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: _col0
+                    outputColumnNames: key
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(_col0)
-                      keys: _col0 (type: string)
+                      aggregations: min(key)
+                      keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1365,24 +1357,20 @@ STAGE PLANS:
                     isSamplingPred: false
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: key (type: string)
-                      outputColumnNames: _col0
+                    Group By Operator
+                      aggregations: min(key)
+                      keys: key (type: string)
+                      mode: hash
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: min(_col0)
-                        keys: _col0 (type: string)
-                        mode: hash
-                        outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: string)
-                          Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                          tag: -1
-                          value expressions: _col1 (type: string)
-                          auto parallelism: false
+                        tag: -1
+                        value expressions: _col1 (type: string)
+                        auto parallelism: false
             Path -> Alias:
 #### A masked pattern was here ####
             Path -> Partition:


[10/50] [abbrv] hive git commit: HIVE-11606: Bucket map joins fail at hash table construction time (Vikram Dixit K, reviewed by Sergey Shelukhin)

Posted by xu...@apache.org.
HIVE-11606: Bucket map joins fail at hash table construction time (Vikram Dixit K, reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9fe8802c
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9fe8802c
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9fe8802c

Branch: refs/heads/spark
Commit: 9fe8802cb83e05c0392b11b8dcfe354fecfda786
Parents: 4ea8e29
Author: vikram <vi...@hortonworks.com>
Authored: Thu Sep 10 13:29:34 2015 -0700
Committer: vikram <vi...@hortonworks.com>
Committed: Thu Sep 10 13:30:24 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/Operator.java    |   3 +-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |  11 +-
 .../clientpositive/bucket_map_join_tez1.q       |  22 ++
 .../spark/bucket_map_join_tez1.q.out            | 226 +++++++++++++++++++
 .../tez/bucket_map_join_tez1.q.out              | 210 +++++++++++++++++
 5 files changed, 469 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9fe8802c/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
index acbe504..92e5446 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
@@ -319,6 +319,7 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
   @SuppressWarnings("unchecked")
   public final void initialize(Configuration hconf, ObjectInspector[] inputOIs)
       throws HiveException {
+    this.done = false;
     if (state == State.INIT) {
       return;
     }
@@ -369,7 +370,7 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
     }
 
     if (isLogInfoEnabled) {
-      LOG.info("Initialization Done " + id + " " + getName());
+      LOG.info("Initialization Done " + id + " " + getName() + " done is reset.");
     }
 
     initializeChildren(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/9fe8802c/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
index 71c766f..757ff5e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
@@ -215,8 +215,15 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
         tableSize /= bucketCount;
       }
     }
-    LOG.info("Mapjoin " + mapJoinOp + ", pos: " + pos + " --> " + parentWork.getName() + " ("
-      + keyCount + " keys estimated from " + rowCount + " rows, " + bucketCount + " buckets)");
+    if (keyCount == 0) {
+      keyCount = 1;
+    }
+    if (tableSize == 0) {
+      tableSize = 1;
+    }
+    LOG.info("Mapjoin " + mapJoinOp + "(bucket map join = )" + joinConf.isBucketMapJoin()
+        + ", pos: " + pos + " --> " + parentWork.getName() + " (" + keyCount
+        + " keys estimated from " + rowCount + " rows, " + bucketCount + " buckets)");
     joinConf.getParentToInput().put(pos, parentWork.getName());
     if (keyCount != Long.MAX_VALUE) {
       joinConf.getParentKeyCounts().put(pos, keyCount);

http://git-wip-us.apache.org/repos/asf/hive/blob/9fe8802c/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q b/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
index 0f9dd6d..8546e78 100644
--- a/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
+++ b/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
@@ -95,3 +95,25 @@ explain select a.key, b.key from tab_part a join tab_part c on a.key = c.key joi
 explain
 select a.key, a.value, b.value
 from tab a join tab_part b on a.key = b.key and a.ds = b.ds;
+
+set hive.mapjoin.hybridgrace.hashtable = false;
+insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin where key = 411;
+
+explain
+select count(*)
+from tab_part a join tab b on a.key = b.key;
+
+select count(*)
+from tab_part a join tab b on a.key = b.key;
+
+set hive.mapjoin.hybridgrace.hashtable = false;
+insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin where key = 411;
+
+explain
+select count(*)
+from tab_part a join tab b on a.key = b.key;
+
+select count(*)
+from tab_part a join tab b on a.key = b.key;

http://git-wip-us.apache.org/repos/asf/hive/blob/9fe8802c/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
index 34ddc90..2c14065 100644
--- a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
@@ -1424,3 +1424,229 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+PREHOOK: query: insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin where key = 411
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcbucket_mapjoin
+PREHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08
+PREHOOK: Output: default@tab@ds=2008-04-08
+POSTHOOK: query: insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin where key = 411
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcbucket_mapjoin
+POSTHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08
+POSTHOOK: Output: default@tab@ds=2008-04-08
+POSTHOOK: Lineage: tab PARTITION(ds=2008-04-08).key SIMPLE [(srcbucket_mapjoin)srcbucket_mapjoin.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: tab PARTITION(ds=2008-04-08).value SIMPLE [(srcbucket_mapjoin)srcbucket_mapjoin.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: explain
+select count(*)
+from tab_part a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select count(*)
+from tab_part a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 key (type: int)
+                        1 key (type: int)
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+      Edges:
+        Reducer 2 <- Map 1 (GROUP, 1)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 key (type: int)
+                        1 key (type: int)
+                      input vertices:
+                        1 Map 3
+                      Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+            Local Work:
+              Map Reduce Local Work
+        Reducer 2 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count()
+                mode: complete
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*)
+from tab_part a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tab
+PREHOOK: Input: default@tab@ds=2008-04-08
+PREHOOK: Input: default@tab_part
+PREHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*)
+from tab_part a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tab
+POSTHOOK: Input: default@tab@ds=2008-04-08
+POSTHOOK: Input: default@tab_part
+POSTHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+1
+PREHOOK: query: insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin where key = 411
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcbucket_mapjoin
+PREHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08
+PREHOOK: Output: default@tab@ds=2008-04-08
+POSTHOOK: query: insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin where key = 411
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcbucket_mapjoin
+POSTHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08
+POSTHOOK: Output: default@tab@ds=2008-04-08
+POSTHOOK: Lineage: tab PARTITION(ds=2008-04-08).key SIMPLE [(srcbucket_mapjoin)srcbucket_mapjoin.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: tab PARTITION(ds=2008-04-08).value SIMPLE [(srcbucket_mapjoin)srcbucket_mapjoin.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: explain
+select count(*)
+from tab_part a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select count(*)
+from tab_part a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 key (type: int)
+                        1 key (type: int)
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+      Edges:
+        Reducer 2 <- Map 1 (GROUP, 1)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 key (type: int)
+                        1 key (type: int)
+                      input vertices:
+                        1 Map 3
+                      Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+            Local Work:
+              Map Reduce Local Work
+        Reducer 2 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count()
+                mode: complete
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*)
+from tab_part a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tab
+PREHOOK: Input: default@tab@ds=2008-04-08
+PREHOOK: Input: default@tab_part
+PREHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*)
+from tab_part a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tab
+POSTHOOK: Input: default@tab@ds=2008-04-08
+POSTHOOK: Input: default@tab_part
+POSTHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+1

http://git-wip-us.apache.org/repos/asf/hive/blob/9fe8802c/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
index 8338672..af5e6e6 100644
--- a/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
+++ b/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
@@ -1350,3 +1350,213 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+PREHOOK: query: insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin where key = 411
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcbucket_mapjoin
+PREHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08
+PREHOOK: Output: default@tab@ds=2008-04-08
+POSTHOOK: query: insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin where key = 411
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcbucket_mapjoin
+POSTHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08
+POSTHOOK: Output: default@tab@ds=2008-04-08
+POSTHOOK: Lineage: tab PARTITION(ds=2008-04-08).key SIMPLE [(srcbucket_mapjoin)srcbucket_mapjoin.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: tab PARTITION(ds=2008-04-08).value SIMPLE [(srcbucket_mapjoin)srcbucket_mapjoin.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: explain
+select count(*)
+from tab_part a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select count(*)
+from tab_part a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Map 1 <- Map 3 (CUSTOM_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 key (type: int)
+                        1 key (type: int)
+                      input vertices:
+                        1 Map 3
+                      Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: key (type: int)
+                      sort order: +
+                      Map-reduce partition columns: key (type: int)
+                      Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count()
+                mode: complete
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*)
+from tab_part a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tab
+PREHOOK: Input: default@tab@ds=2008-04-08
+PREHOOK: Input: default@tab_part
+PREHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*)
+from tab_part a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tab
+POSTHOOK: Input: default@tab@ds=2008-04-08
+POSTHOOK: Input: default@tab_part
+POSTHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+1
+PREHOOK: query: insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin where key = 411
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcbucket_mapjoin
+PREHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08
+PREHOOK: Output: default@tab@ds=2008-04-08
+POSTHOOK: query: insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin where key = 411
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcbucket_mapjoin
+POSTHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08
+POSTHOOK: Output: default@tab@ds=2008-04-08
+POSTHOOK: Lineage: tab PARTITION(ds=2008-04-08).key SIMPLE [(srcbucket_mapjoin)srcbucket_mapjoin.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: tab PARTITION(ds=2008-04-08).value SIMPLE [(srcbucket_mapjoin)srcbucket_mapjoin.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: explain
+select count(*)
+from tab_part a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select count(*)
+from tab_part a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Map 1 <- Map 3 (CUSTOM_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 key (type: int)
+                        1 key (type: int)
+                      input vertices:
+                        1 Map 3
+                      Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: key (type: int)
+                      sort order: +
+                      Map-reduce partition columns: key (type: int)
+                      Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count()
+                mode: complete
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*)
+from tab_part a join tab b on a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tab
+PREHOOK: Input: default@tab@ds=2008-04-08
+PREHOOK: Input: default@tab_part
+PREHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*)
+from tab_part a join tab b on a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tab
+POSTHOOK: Input: default@tab@ds=2008-04-08
+POSTHOOK: Input: default@tab_part
+POSTHOOK: Input: default@tab_part@ds=2008-04-08
+#### A masked pattern was here ####
+1


[18/50] [abbrv] hive git commit: HIVE-11763: Use * instead of sum(hash(*)) on Parquet predicate (PPD) integration tests (Sergio Pena, reviewed by Ferdinand Xu)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/results/clientpositive/parquet_ppd_date.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_ppd_date.q.out b/ql/src/test/results/clientpositive/parquet_ppd_date.q.out
index aba302e..60c9a59 100644
--- a/ql/src/test/results/clientpositive/parquet_ppd_date.q.out
+++ b/ql/src/test/results/clientpositive/parquet_ppd_date.q.out
@@ -6,11 +6,11 @@ POSTHOOK: query: create table newtypestbl(c char(10), v varchar(10), d decimal(5
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@newtypestbl
-PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl
+PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@newtypestbl
-POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl
+POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@newtypestbl
@@ -19,283 +19,417 @@ POSTHOOK: Lineage: newtypestbl.d EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.da EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.v EXPRESSION []
 PREHOOK: query: -- date data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where da='1970-02-20'
+select * from newtypestbl where da='1970-02-20'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
 POSTHOOK: query: -- date data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where da='1970-02-20'
+select * from newtypestbl where da='1970-02-20'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da='1970-02-20'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da='1970-02-20'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da='1970-02-20'
+POSTHOOK: query: select * from newtypestbl where da='1970-02-20'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da= date '1970-02-20'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da= date '1970-02-20'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da= date '1970-02-20'
+POSTHOOK: query: select * from newtypestbl where da= date '1970-02-20'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as date)
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da=cast('1970-02-20' as date)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as date)
+POSTHOOK: query: select * from newtypestbl where da=cast('1970-02-20' as date)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as date)
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da=cast('1970-02-20' as date)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as date)
+POSTHOOK: query: select * from newtypestbl where da=cast('1970-02-20' as date)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as varchar(20))
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da=cast('1970-02-20' as varchar(20))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as varchar(20))
+POSTHOOK: query: select * from newtypestbl where da=cast('1970-02-20' as varchar(20))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as varchar(20))
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da=cast('1970-02-20' as varchar(20))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da=cast('1970-02-20' as varchar(20))
+POSTHOOK: query: select * from newtypestbl where da=cast('1970-02-20' as varchar(20))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da!='1970-02-20'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da!='1970-02-20'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da!='1970-02-20'
+POSTHOOK: query: select * from newtypestbl where da!='1970-02-20'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da!='1970-02-20'
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where da!='1970-02-20'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da!='1970-02-20'
+POSTHOOK: query: select * from newtypestbl where da!='1970-02-20'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-27'
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where da<'1970-02-27'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-27'
+POSTHOOK: query: select * from newtypestbl where da<'1970-02-27'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-27'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da<'1970-02-27'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-27'
+POSTHOOK: query: select * from newtypestbl where da<'1970-02-27'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-29'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da<'1970-02-29'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-29'
+POSTHOOK: query: select * from newtypestbl where da<'1970-02-29'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-29'
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da<'1970-02-29'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-29'
+POSTHOOK: query: select * from newtypestbl where da<'1970-02-29'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-15'
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da<'1970-02-15'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-15'
+POSTHOOK: query: select * from newtypestbl where da<'1970-02-15'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-15'
+PREHOOK: query: select * from newtypestbl where da<'1970-02-15'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da<'1970-02-15'
+POSTHOOK: query: select * from newtypestbl where da<'1970-02-15'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da<='1970-02-20'
+PREHOOK: query: select * from newtypestbl where da<='1970-02-20'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da<='1970-02-20'
+POSTHOOK: query: select * from newtypestbl where da<='1970-02-20'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da<='1970-02-20'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da<='1970-02-20'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da<='1970-02-20'
+POSTHOOK: query: select * from newtypestbl where da<='1970-02-20'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da<='1970-02-27'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da<='1970-02-27'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da<='1970-02-27'
+POSTHOOK: query: select * from newtypestbl where da<='1970-02-27'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da<='1970-02-27'
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da<='1970-02-27'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da<='1970-02-27'
+POSTHOOK: query: select * from newtypestbl where da<='1970-02-27'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date))
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date))
+POSTHOOK: query: select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date))
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date))
+POSTHOOK: query: select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-27' as date))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date))
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date))
+POSTHOOK: query: select * from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date))
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date))
+POSTHOOK: query: select * from newtypestbl where da in (cast('1970-02-20' as date), cast('1970-02-27' as date))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date))
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date))
+POSTHOOK: query: select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date))
+PREHOOK: query: select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date))
+POSTHOOK: query: select * from newtypestbl where da in (cast('1970-02-21' as date), cast('1970-02-22' as date))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-22'
+PREHOOK: query: select * from newtypestbl where da between '1970-02-19' and '1970-02-22'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-22'
+POSTHOOK: query: select * from newtypestbl where da between '1970-02-19' and '1970-02-22'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-22'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da between '1970-02-19' and '1970-02-22'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-22'
+POSTHOOK: query: select * from newtypestbl where da between '1970-02-19' and '1970-02-22'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-28'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da between '1970-02-19' and '1970-02-28'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-28'
+POSTHOOK: query: select * from newtypestbl where da between '1970-02-19' and '1970-02-28'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-28'
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da between '1970-02-19' and '1970-02-28'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-19' and '1970-02-28'
+POSTHOOK: query: select * from newtypestbl where da between '1970-02-19' and '1970-02-28'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-18' and '1970-02-19'
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where da between '1970-02-18' and '1970-02-19'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-18' and '1970-02-19'
+POSTHOOK: query: select * from newtypestbl where da between '1970-02-18' and '1970-02-19'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-18' and '1970-02-19'
+PREHOOK: query: select * from newtypestbl where da between '1970-02-18' and '1970-02-19'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where da between '1970-02-18' and '1970-02-19'
+POSTHOOK: query: select * from newtypestbl where da between '1970-02-18' and '1970-02-19'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/66fb9601/ql/src/test/results/clientpositive/parquet_ppd_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_ppd_decimal.q.out b/ql/src/test/results/clientpositive/parquet_ppd_decimal.q.out
index 9e48df8..ec603eb 100644
--- a/ql/src/test/results/clientpositive/parquet_ppd_decimal.q.out
+++ b/ql/src/test/results/clientpositive/parquet_ppd_decimal.q.out
@@ -6,11 +6,11 @@ POSTHOOK: query: create table newtypestbl(c char(10), v varchar(10), d decimal(5
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@newtypestbl
-PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl
+PREHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@newtypestbl
-POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl
+POSTHOOK: query: insert overwrite table newtypestbl select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2 limit 10) uniontbl
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@newtypestbl
@@ -19,472 +19,750 @@ POSTHOOK: Lineage: newtypestbl.d EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.da EXPRESSION []
 POSTHOOK: Lineage: newtypestbl.v EXPRESSION []
 PREHOOK: query: -- decimal data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where d=0.22
+select * from newtypestbl where d=0.22
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
 POSTHOOK: query: -- decimal data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
-select sum(hash(*)) from newtypestbl where d=0.22
+select * from newtypestbl where d=0.22
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d=0.22
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d=0.22
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d=0.22
+POSTHOOK: query: select * from newtypestbl where d=0.22
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d='0.22'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d='0.22'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d='0.22'
+POSTHOOK: query: select * from newtypestbl where d='0.22'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d='0.22'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d='0.22'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d='0.22'
+POSTHOOK: query: select * from newtypestbl where d='0.22'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d=cast('0.22' as float)
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d=cast('0.22' as float)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d=cast('0.22' as float)
+POSTHOOK: query: select * from newtypestbl where d=cast('0.22' as float)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d=cast('0.22' as float)
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d=cast('0.22' as float)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d=cast('0.22' as float)
+POSTHOOK: query: select * from newtypestbl where d=cast('0.22' as float)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d!=0.22
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d!=0.22
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d!=0.22
+POSTHOOK: query: select * from newtypestbl where d!=0.22
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d!=0.22
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where d!=0.22
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d!=0.22
+POSTHOOK: query: select * from newtypestbl where d!=0.22
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d!='0.22'
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where d!='0.22'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d!='0.22'
+POSTHOOK: query: select * from newtypestbl where d!='0.22'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d!='0.22'
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where d!='0.22'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d!='0.22'
+POSTHOOK: query: select * from newtypestbl where d!='0.22'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d!=cast('0.22' as float)
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where d!=cast('0.22' as float)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d!=cast('0.22' as float)
+POSTHOOK: query: select * from newtypestbl where d!=cast('0.22' as float)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d!=cast('0.22' as float)
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where d!=cast('0.22' as float)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d!=cast('0.22' as float)
+POSTHOOK: query: select * from newtypestbl where d!=cast('0.22' as float)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<11.22
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where d<11.22
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<11.22
+POSTHOOK: query: select * from newtypestbl where d<11.22
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<11.22
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<11.22
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<11.22
+POSTHOOK: query: select * from newtypestbl where d<11.22
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<'11.22'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<'11.22'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<'11.22'
+POSTHOOK: query: select * from newtypestbl where d<'11.22'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<'11.22'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<'11.22'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<'11.22'
+POSTHOOK: query: select * from newtypestbl where d<'11.22'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<cast('11.22' as float)
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<cast('11.22' as float)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<cast('11.22' as float)
+POSTHOOK: query: select * from newtypestbl where d<cast('11.22' as float)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<cast('11.22' as float)
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<cast('11.22' as float)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<cast('11.22' as float)
+POSTHOOK: query: select * from newtypestbl where d<cast('11.22' as float)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<1
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<1
+POSTHOOK: query: select * from newtypestbl where d<1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<1
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<1
+POSTHOOK: query: select * from newtypestbl where d<1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<=11.22
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=11.22
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<=11.22
+POSTHOOK: query: select * from newtypestbl where d<=11.22
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<=11.22
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=11.22
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<=11.22
+POSTHOOK: query: select * from newtypestbl where d<=11.22
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<='11.22'
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<='11.22'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<='11.22'
+POSTHOOK: query: select * from newtypestbl where d<='11.22'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<='11.22'
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<='11.22'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<='11.22'
+POSTHOOK: query: select * from newtypestbl where d<='11.22'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<=cast('11.22' as float)
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=cast('11.22' as float)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<=cast('11.22' as float)
+POSTHOOK: query: select * from newtypestbl where d<=cast('11.22' as float)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<=cast('11.22' as float)
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=cast('11.22' as float)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<=cast('11.22' as float)
+POSTHOOK: query: select * from newtypestbl where d<=cast('11.22' as float)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<=cast('11.22' as decimal)
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=cast('11.22' as decimal)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<=cast('11.22' as decimal)
+POSTHOOK: query: select * from newtypestbl where d<=cast('11.22' as decimal)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<=cast('11.22' as decimal)
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=cast('11.22' as decimal)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<=cast('11.22' as decimal)
+POSTHOOK: query: select * from newtypestbl where d<=cast('11.22' as decimal)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<=11.22BD
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=11.22BD
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<=11.22BD
+POSTHOOK: query: select * from newtypestbl where d<=11.22BD
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<=11.22BD
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=11.22BD
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<=11.22BD
+POSTHOOK: query: select * from newtypestbl where d<=11.22BD
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<=12
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=12
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<=12
+POSTHOOK: query: select * from newtypestbl where d<=12
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d<=12
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d<=12
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d<=12
+POSTHOOK: query: select * from newtypestbl where d<=12
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.22', '1.0')
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.22', '1.0')
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.22', '1.0')
+POSTHOOK: query: select * from newtypestbl where d in ('0.22', '1.0')
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.22', '1.0')
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.22', '1.0')
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.22', '1.0')
+POSTHOOK: query: select * from newtypestbl where d in ('0.22', '1.0')
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.22', '11.22')
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.22', '11.22')
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.22', '11.22')
+POSTHOOK: query: select * from newtypestbl where d in ('0.22', '11.22')
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.22', '11.22')
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.22', '11.22')
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.22', '11.22')
+POSTHOOK: query: select * from newtypestbl where d in ('0.22', '11.22')
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', '1.0')
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.9', '1.0')
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', '1.0')
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', '1.0')
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', '1.0')
+PREHOOK: query: select * from newtypestbl where d in ('0.9', '1.0')
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', '1.0')
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', '1.0')
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-NULL
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22)
+PREHOOK: query: select * from newtypestbl where d in ('0.9', 0.22)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22)
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', 0.22)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22)
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.9', 0.22)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22)
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', 0.22)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float))
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float))
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float))
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float))
+POSTHOOK: query: select * from newtypestbl where d in ('0.9', 0.22, cast('11.22' as float))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and 1
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and 1
+POSTHOOK: query: select * from newtypestbl where d between 0 and 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and 1
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and 1
+POSTHOOK: query: select * from newtypestbl where d between 0 and 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and 1000
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and 1000
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and 1000
+POSTHOOK: query: select * from newtypestbl where d between 0 and 1000
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and 1000
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and 1000
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and 1000
+POSTHOOK: query: select * from newtypestbl where d between 0 and 1000
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-81475875500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and '2.0'
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+hello     	world	11.22	1970-02-27
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and '2.0'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and '2.0'
+POSTHOOK: query: select * from newtypestbl where d between 0 and '2.0'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and '2.0'
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and '2.0'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and '2.0'
+POSTHOOK: query: select * from newtypestbl where d between 0 and '2.0'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and cast(3 as float)
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and cast(3 as float)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and cast(3 as float)
+POSTHOOK: query: select * from newtypestbl where d between 0 and cast(3 as float)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and cast(3 as float)
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 0 and cast(3 as float)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d between 0 and cast(3 as float)
+POSTHOOK: query: select * from newtypestbl where d between 0 and cast(3 as float)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
--252951929000
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d between 1 and cast(30 as char(10))
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+apple     	bee	0.22	1970-02-20
+PREHOOK: query: select * from newtypestbl where d between 1 and cast(30 as char(10))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d between 1 and cast(30 as char(10))
+POSTHOOK: query: select * from newtypestbl where d between 1 and cast(30 as char(10))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
-PREHOOK: query: select sum(hash(*)) from newtypestbl where d between 1 and cast(30 as char(10))
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+PREHOOK: query: select * from newtypestbl where d between 1 and cast(30 as char(10))
 PREHOOK: type: QUERY
 PREHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-POSTHOOK: query: select sum(hash(*)) from newtypestbl where d between 1 and cast(30 as char(10))
+POSTHOOK: query: select * from newtypestbl where d between 1 and cast(30 as char(10))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@newtypestbl
 #### A masked pattern was here ####
-334427804500
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27
+hello     	world	11.22	1970-02-27


[14/50] [abbrv] hive git commit: HIVE-11771: Parquet timestamp conversion errors (Jimmy, reviewed by Szehon)

Posted by xu...@apache.org.
HIVE-11771: Parquet timestamp conversion errors (Jimmy, reviewed by Szehon)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1e97b161
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1e97b161
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1e97b161

Branch: refs/heads/spark
Commit: 1e97b16181941f8c21684f4b7a4958b890ef7738
Parents: b1fffd5
Author: Jimmy Xiang <jx...@cloudera.com>
Authored: Wed Sep 9 13:26:06 2015 -0700
Committer: Jimmy Xiang <jx...@cloudera.com>
Committed: Sat Sep 12 14:43:14 2015 -0700

----------------------------------------------------------------------
 .../ql/io/parquet/timestamp/NanoTimeUtils.java  | 23 +++++++++---
 .../serde/TestParquetTimestampUtils.java        | 38 +++++++++++++++++++-
 2 files changed, 56 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1e97b161/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
index 59c9b4a..aace48e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
@@ -15,6 +15,7 @@ package org.apache.hadoop.hive.ql.io.parquet.timestamp;
 
 import java.sql.Timestamp;
 import java.util.Calendar;
+import java.util.GregorianCalendar;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 
@@ -28,6 +29,7 @@ public class NanoTimeUtils {
    static final long NANOS_PER_HOUR = TimeUnit.HOURS.toNanos(1);
    static final long NANOS_PER_MINUTE = TimeUnit.MINUTES.toNanos(1);
    static final long NANOS_PER_SECOND = TimeUnit.SECONDS.toNanos(1);
+   static final long NANOS_PER_DAY = TimeUnit.DAYS.toNanos(1);
 
    private static final ThreadLocal<Calendar> parquetGMTCalendar = new ThreadLocal<Calendar>();
    private static final ThreadLocal<Calendar> parquetLocalCalendar = new ThreadLocal<Calendar>();
@@ -48,14 +50,20 @@ public class NanoTimeUtils {
    }
 
    private static Calendar getCalendar(boolean skipConversion) {
-     return skipConversion ? getLocalCalendar() : getGMTCalendar();
+     Calendar calendar = skipConversion ? getLocalCalendar() : getGMTCalendar();
+     calendar.clear(); // Reset all fields before reusing this instance
+     return calendar;
    }
 
    public static NanoTime getNanoTime(Timestamp ts, boolean skipConversion) {
 
      Calendar calendar = getCalendar(skipConversion);
      calendar.setTime(ts);
-     JDateTime jDateTime = new JDateTime(calendar.get(Calendar.YEAR),
+     int year = calendar.get(Calendar.YEAR);
+     if (calendar.get(Calendar.ERA) == GregorianCalendar.BC) {
+       year = 1 - year;
+     }
+     JDateTime jDateTime = new JDateTime(year,
        calendar.get(Calendar.MONTH) + 1,  //java calendar index starting at 1.
        calendar.get(Calendar.DAY_OF_MONTH));
      int days = jDateTime.getJulianDayNumber();
@@ -74,13 +82,20 @@ public class NanoTimeUtils {
      int julianDay = nt.getJulianDay();
      long nanosOfDay = nt.getTimeOfDayNanos();
 
+     long remainder = nanosOfDay;
+     julianDay += remainder / NANOS_PER_DAY;
+     remainder %= NANOS_PER_DAY;
+     if (remainder < 0) {
+       remainder += NANOS_PER_DAY;
+       julianDay--;
+     }
+
      JDateTime jDateTime = new JDateTime((double) julianDay);
      Calendar calendar = getCalendar(skipConversion);
      calendar.set(Calendar.YEAR, jDateTime.getYear());
-     calendar.set(Calendar.MONTH, jDateTime.getMonth() - 1); //java calender index starting at 1.
+     calendar.set(Calendar.MONTH, jDateTime.getMonth() - 1); //java calendar index starting at 1.
      calendar.set(Calendar.DAY_OF_MONTH, jDateTime.getDay());
 
-     long remainder = nanosOfDay;
      int hour = (int) (remainder / (NANOS_PER_HOUR));
      remainder = remainder % (NANOS_PER_HOUR);
      int minutes = (int) (remainder / (NANOS_PER_MINUTE));

http://git-wip-us.apache.org/repos/asf/hive/blob/1e97b161/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
index 510ffd1..ec6def5 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
@@ -16,7 +16,9 @@ package org.apache.hadoop.hive.ql.io.parquet.serde;
 import java.sql.Timestamp;
 import java.util.Calendar;
 import java.util.Date;
+import java.util.GregorianCalendar;
 import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
 
 import junit.framework.Assert;
 import junit.framework.TestCase;
@@ -74,7 +76,36 @@ public class TestParquetTimestampUtils extends TestCase {
     Timestamp ts2Fetched = NanoTimeUtils.getTimestamp(nt2, false);
     Assert.assertEquals(ts2Fetched, ts2);
     Assert.assertEquals(nt2.getJulianDay() - nt1.getJulianDay(), 30);
-  }
+
+    //check if 1464305 Julian Days between Jan 1, 2005 BC and Jan 31, 2005.
+    cal1 = Calendar.getInstance();
+    cal1.set(Calendar.ERA,  GregorianCalendar.BC);
+    cal1.set(Calendar.YEAR,  2005);
+    cal1.set(Calendar.MONTH, Calendar.JANUARY);
+    cal1.set(Calendar.DAY_OF_MONTH, 1);
+    cal1.set(Calendar.HOUR_OF_DAY, 0);
+    cal1.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+    ts1 = new Timestamp(cal1.getTimeInMillis());
+    nt1 = NanoTimeUtils.getNanoTime(ts1, false);
+
+    ts1Fetched = NanoTimeUtils.getTimestamp(nt1, false);
+    Assert.assertEquals(ts1Fetched, ts1);
+
+    cal2 = Calendar.getInstance();
+    cal2.set(Calendar.YEAR,  2005);
+    cal2.set(Calendar.MONTH, Calendar.JANUARY);
+    cal2.set(Calendar.DAY_OF_MONTH, 31);
+    cal2.set(Calendar.HOUR_OF_DAY, 0);
+    cal2.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+    ts2 = new Timestamp(cal2.getTimeInMillis());
+    nt2 = NanoTimeUtils.getNanoTime(ts2, false);
+
+    ts2Fetched = NanoTimeUtils.getTimestamp(nt2, false);
+    Assert.assertEquals(ts2Fetched, ts2);
+    Assert.assertEquals(nt2.getJulianDay() - nt1.getJulianDay(), 1464305);
+}
 
   public void testNanos() {
     //case 1: 01:01:01.0000000001
@@ -136,6 +167,11 @@ public class TestParquetTimestampUtils extends TestCase {
     NanoTime n1 = NanoTimeUtils.getNanoTime(ts1, false);
 
     Assert.assertEquals(n2.getTimeOfDayNanos() - n1.getTimeOfDayNanos(), 600000000009L);
+
+    NanoTime n3 = new NanoTime(n1.getJulianDay() - 1, n1.getTimeOfDayNanos() + TimeUnit.DAYS.toNanos(1));
+    Assert.assertEquals(ts1, NanoTimeUtils.getTimestamp(n3, false));
+    n3 = new NanoTime(n1.getJulianDay() + 3, n1.getTimeOfDayNanos() - TimeUnit.DAYS.toNanos(3));
+    Assert.assertEquals(ts1, NanoTimeUtils.getTimestamp(n3, false));
   }
 
   public void testTimezone() {


[36/50] [abbrv] hive git commit: HIVE-11678 : Add AggregateProjectMergeRule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_outer_join2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_outer_join2.q.out b/ql/src/test/results/clientpositive/vector_outer_join2.q.out
index c649883..236fa18 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join2.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join2.q.out
@@ -193,14 +193,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           TableScan
             alias: c
             Statistics: Num rows: 20 Data size: 4182 Basic stats: COMPLETE Column stats: NONE
@@ -212,7 +212,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           TableScan
             alias: c
             Statistics: Num rows: 20 Data size: 4182 Basic stats: COMPLETE Column stats: NONE
@@ -251,19 +251,15 @@ STAGE PLANS:
                     1 _col0 (type: bigint)
                   outputColumnNames: _col1
                   Statistics: Num rows: 24 Data size: 5060 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: _col1 (type: bigint)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 24 Data size: 5060 Basic stats: COMPLETE Column stats: NONE
-                    Group By Operator
-                      aggregations: count(), sum(_col0)
-                      mode: hash
-                      outputColumnNames: _col0, _col1
+                  Group By Operator
+                    aggregations: count(), sum(_col1)
+                    mode: hash
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
                       Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        sort order: 
-                        Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: bigint), _col1 (type: bigint)
+                      value expressions: _col0 (type: bigint), _col1 (type: bigint)
       Local Work:
         Map Reduce Local Work
       Execution mode: vectorized

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_outer_join3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_outer_join3.q.out b/ql/src/test/results/clientpositive/vector_outer_join3.q.out
index 502f2d3..821a642 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join3.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join3.q.out
@@ -193,14 +193,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           TableScan
             alias: c
             Statistics: Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE
@@ -212,7 +212,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           TableScan
             alias: c
             Statistics: Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE
@@ -334,14 +334,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           TableScan
             alias: c
             Statistics: Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE
@@ -353,7 +353,7 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           TableScan
             alias: c
             Statistics: Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE
@@ -475,14 +475,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           TableScan
             alias: c
             Statistics: Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE
@@ -494,7 +494,7 @@ STAGE PLANS:
                 keys:
                   0 _col3 (type: string), _col1 (type: bigint)
                   1 _col1 (type: string), _col0 (type: bigint)
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           TableScan
             alias: c
             Statistics: Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_outer_join4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_outer_join4.q.out b/ql/src/test/results/clientpositive/vector_outer_join4.q.out
index 50ae497..fad7685 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join4.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join4.q.out
@@ -859,14 +859,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:c 
+        $hdt$_1:c 
           TableScan
             alias: c
             Statistics: Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE
@@ -878,7 +878,7 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: int)
                   1 _col0 (type: int)
-        $hdt$_0:$hdt$_2:c 
+        $hdt$_2:c 
           TableScan
             alias: c
             Statistics: Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_outer_join5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_outer_join5.q.out b/ql/src/test/results/clientpositive/vector_outer_join5.q.out
index bbe8ba1..2cb80b6 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join5.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join5.q.out
@@ -85,11 +85,11 @@ STAGE PLANS:
   Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:st 
+        $hdt$_1:st 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:st 
+        $hdt$_1:st 
           TableScan
             alias: st
             Statistics: Num rows: 100 Data size: 392 Basic stats: COMPLETE Column stats: NONE
@@ -193,11 +193,11 @@ STAGE PLANS:
   Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           TableScan
             alias: sm
             Statistics: Num rows: 100 Data size: 392 Basic stats: COMPLETE Column stats: NONE
@@ -307,11 +307,11 @@ STAGE PLANS:
   Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           TableScan
             alias: sm
             Statistics: Num rows: 100 Data size: 392 Basic stats: COMPLETE Column stats: NONE
@@ -421,11 +421,11 @@ STAGE PLANS:
   Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           TableScan
             alias: sm
             Statistics: Num rows: 100 Data size: 392 Basic stats: COMPLETE Column stats: NONE
@@ -539,14 +539,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_2:s 
+        $hdt$_2:s 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           TableScan
             alias: sm
             Statistics: Num rows: 100 Data size: 392 Basic stats: COMPLETE Column stats: NONE
@@ -558,7 +558,7 @@ STAGE PLANS:
                 keys:
                   0 UDFToLong(_col1) (type: bigint)
                   1 (_col0 pmod UDFToLong(8)) (type: bigint)
-        $hdt$_0:$hdt$_2:s 
+        $hdt$_2:s 
           TableScan
             alias: s
             Statistics: Num rows: 6058 Data size: 48464 Basic stats: COMPLETE Column stats: NONE
@@ -732,11 +732,11 @@ STAGE PLANS:
   Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:st 
+        $hdt$_1:st 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:st 
+        $hdt$_1:st 
           TableScan
             alias: st
             Statistics: Num rows: 100 Data size: 392 Basic stats: COMPLETE Column stats: NONE
@@ -840,11 +840,11 @@ STAGE PLANS:
   Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           TableScan
             alias: sm
             Statistics: Num rows: 100 Data size: 392 Basic stats: COMPLETE Column stats: NONE
@@ -954,11 +954,11 @@ STAGE PLANS:
   Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           TableScan
             alias: sm
             Statistics: Num rows: 100 Data size: 392 Basic stats: COMPLETE Column stats: NONE
@@ -1068,11 +1068,11 @@ STAGE PLANS:
   Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           TableScan
             alias: sm
             Statistics: Num rows: 100 Data size: 392 Basic stats: COMPLETE Column stats: NONE
@@ -1186,14 +1186,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_2:s 
+        $hdt$_2:s 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_1:sm 
+        $hdt$_1:sm 
           TableScan
             alias: sm
             Statistics: Num rows: 100 Data size: 392 Basic stats: COMPLETE Column stats: NONE
@@ -1205,7 +1205,7 @@ STAGE PLANS:
                 keys:
                   0 UDFToLong(_col1) (type: bigint)
                   1 (_col0 pmod UDFToLong(8)) (type: bigint)
-        $hdt$_0:$hdt$_2:s 
+        $hdt$_2:s 
           TableScan
             alias: s
             Statistics: Num rows: 6058 Data size: 48464 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_partition_diff_num_cols.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_partition_diff_num_cols.q.out b/ql/src/test/results/clientpositive/vector_partition_diff_num_cols.q.out
index 0437ff6..f483614 100644
--- a/ql/src/test/results/clientpositive/vector_partition_diff_num_cols.q.out
+++ b/ql/src/test/results/clientpositive/vector_partition_diff_num_cols.q.out
@@ -95,10 +95,10 @@ STAGE PLANS:
             Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: inv_quantity_on_hand (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: inv_quantity_on_hand
               Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(inv_quantity_on_hand)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -214,10 +214,10 @@ STAGE PLANS:
             Statistics: Num rows: 200 Data size: 11876 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: inv_quantity_on_hand (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: inv_quantity_on_hand
               Statistics: Num rows: 200 Data size: 11876 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(inv_quantity_on_hand)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -334,10 +334,10 @@ STAGE PLANS:
             Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: inv_quantity_on_hand (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: inv_quantity_on_hand
               Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(inv_quantity_on_hand)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -435,10 +435,10 @@ STAGE PLANS:
             Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: inv_quantity_on_hand (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: inv_quantity_on_hand
               Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(inv_quantity_on_hand)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -542,10 +542,10 @@ STAGE PLANS:
             Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: inv_quantity_on_hand (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: inv_quantity_on_hand
               Statistics: Num rows: 200 Data size: 3176 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(inv_quantity_on_hand)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_partitioned_date_time.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_partitioned_date_time.q.out b/ql/src/test/results/clientpositive/vector_partitioned_date_time.q.out
index 83e7f19..a2762a3 100644
--- a/ql/src/test/results/clientpositive/vector_partitioned_date_time.q.out
+++ b/ql/src/test/results/clientpositive/vector_partitioned_date_time.q.out
@@ -376,11 +376,11 @@ STAGE PLANS:
             Statistics: Num rows: 137 Data size: 39456 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: fl_date (type: date)
-              outputColumnNames: _col0
+              outputColumnNames: fl_date
               Statistics: Num rows: 137 Data size: 39456 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: date)
+                keys: fl_date (type: date)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 137 Data size: 39456 Basic stats: COMPLETE Column stats: NONE
@@ -1153,11 +1153,11 @@ STAGE PLANS:
             Statistics: Num rows: 137 Data size: 31776 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: fl_date (type: date)
-              outputColumnNames: _col0
+              outputColumnNames: fl_date
               Statistics: Num rows: 137 Data size: 31776 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: date)
+                keys: fl_date (type: date)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 137 Data size: 31776 Basic stats: COMPLETE Column stats: NONE
@@ -1954,11 +1954,11 @@ STAGE PLANS:
             Statistics: Num rows: 137 Data size: 33968 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: fl_time (type: timestamp)
-              outputColumnNames: _col0
+              outputColumnNames: fl_time
               Statistics: Num rows: 137 Data size: 33968 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                keys: _col0 (type: timestamp)
+                keys: fl_time (type: timestamp)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 137 Data size: 33968 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out b/ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out
index b38add3..77dc175 100644
--- a/ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out
+++ b/ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out
@@ -39,22 +39,18 @@ STAGE PLANS:
             Filter Operator
               predicate: (cdecimal1 is not null and cdecimal2 is not null) (type: boolean)
               Statistics: Num rows: 1526 Data size: 360136 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: cint (type: int), cdouble (type: double), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14))
-                outputColumnNames: _col0, _col1, _col2, _col3
+              Group By Operator
+                aggregations: min(cdecimal1)
+                keys: cint (type: int), cdouble (type: double), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14))
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1526 Data size: 360136 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: min(_col2)
-                  keys: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14))
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                Reduce Output Operator
+                  key expressions: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14))
+                  sort order: ++++
+                  Map-reduce partition columns: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14))
                   Statistics: Num rows: 1526 Data size: 360136 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14))
-                    sort order: ++++
-                    Map-reduce partition columns: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14))
-                    Statistics: Num rows: 1526 Data size: 360136 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col4 (type: decimal(20,10))
+                  value expressions: _col4 (type: decimal(20,10))
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorization_0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_0.q.out b/ql/src/test/results/clientpositive/vectorization_0.q.out
index 89163cd..3671055 100644
--- a/ql/src/test/results/clientpositive/vectorization_0.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_0.q.out
@@ -130,10 +130,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctinyint (type: tinyint)
-              outputColumnNames: _col0
+              outputColumnNames: ctinyint
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(ctinyint)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -443,10 +443,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: cbigint (type: bigint)
-              outputColumnNames: _col0
+              outputColumnNames: cbigint
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(cbigint)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -756,10 +756,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: cfloat (type: float)
-              outputColumnNames: _col0
+              outputColumnNames: cfloat
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(cfloat)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -1014,10 +1014,10 @@ STAGE PLANS:
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: cbigint (type: bigint), cfloat (type: float), ctinyint (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2
+                outputColumnNames: cbigint, cfloat, ctinyint
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: avg(_col0), stddev_pop(_col0), var_samp(_col0), count(), sum(_col1), min(_col2)
+                  aggregations: avg(cbigint), stddev_pop(cbigint), var_samp(cbigint), count(), sum(cfloat), min(ctinyint)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                   Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorization_13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_13.q.out b/ql/src/test/results/clientpositive/vectorization_13.q.out
index 95cb09a..38966d1 100644
--- a/ql/src/test/results/clientpositive/vectorization_13.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_13.q.out
@@ -84,31 +84,31 @@ STAGE PLANS:
               predicate: (((cfloat < 3569.0) and (10.175 >= cdouble) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > 11.0) and (UDFToDouble(ctimestamp2) <> 12.0) and (UDFToDouble(ctinyint) < 9763215.5639))) (type: boolean)
               Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                expressions: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
+                outputColumnNames: ctinyint, cfloat, cstring1, ctimestamp1, cboolean1
                 Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: max(_col1), sum(_col3), stddev_pop(_col3), stddev_pop(_col1), max(_col3), min(_col1)
-                  keys: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                  aggregations: max(ctinyint), sum(cfloat), stddev_pop(cfloat), stddev_pop(ctinyint), max(cfloat), min(ctinyint)
+                  keys: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                   Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
-                    key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                    key expressions: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                     sort order: +++++
-                    Map-reduce partition columns: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                    Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                     Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: struct<count:bigint,sum:double,variance:double>), _col9 (type: float), _col10 (type: tinyint)
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           aggregations: max(VALUE._col0), sum(VALUE._col1), stddev_pop(VALUE._col2), stddev_pop(VALUE._col3), max(VALUE._col4), min(VALUE._col5)
-          keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
+          keys: KEY._col0 (type: tinyint), KEY._col1 (type: float), KEY._col2 (type: string), KEY._col3 (type: timestamp), KEY._col4 (type: boolean)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
           Statistics: Num rows: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col3)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col1) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col1) + _col5))) / UDFToDouble(_col1)) (type: double), _col10 (type: tinyint)
+            expressions: _col4 (type: boolean), _col0 (type: tinyint), _col3 (type: timestamp), _col1 (type: float), _col2 (type: string), (- _col0) (type: tinyint), _col5 (type: tinyint), ((- _col0) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col0) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col1)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col0) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col0) + _col5))) / UDFToDouble(_col0)) (type: double), _col10 (type: tinyint)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
             Statistics: Num rows: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
@@ -340,31 +340,31 @@ STAGE PLANS:
               predicate: (((cfloat < 3569.0) and (10.175 >= cdouble) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -1.388) and (UDFToDouble(ctimestamp2) <> -1.3359999999999999) and (UDFToDouble(ctinyint) < 9763215.5639))) (type: boolean)
               Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                expressions: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
+                outputColumnNames: ctinyint, cfloat, cstring1, ctimestamp1, cboolean1
                 Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: max(_col1), sum(_col3), stddev_pop(_col3), stddev_pop(_col1), max(_col3), min(_col1)
-                  keys: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                  aggregations: max(ctinyint), sum(cfloat), stddev_pop(cfloat), stddev_pop(ctinyint), max(cfloat), min(ctinyint)
+                  keys: ctinyint (type: tinyint), cfloat (type: float), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                   Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
-                    key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                    key expressions: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                     sort order: +++++
-                    Map-reduce partition columns: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
+                    Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: float), _col2 (type: string), _col3 (type: timestamp), _col4 (type: boolean)
                     Statistics: Num rows: 2730 Data size: 586959 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: struct<count:bigint,sum:double,variance:double>), _col9 (type: float), _col10 (type: tinyint)
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           aggregations: max(VALUE._col0), sum(VALUE._col1), stddev_pop(VALUE._col2), stddev_pop(VALUE._col3), max(VALUE._col4), min(VALUE._col5)
-          keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
+          keys: KEY._col0 (type: tinyint), KEY._col1 (type: float), KEY._col2 (type: string), KEY._col3 (type: timestamp), KEY._col4 (type: boolean)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
           Statistics: Num rows: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col3)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col1) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col1) + _col5))) / UDFToDouble(_col1)) (type: double), _col10 (type: tinyint)
+            expressions: _col4 (type: boolean), _col0 (type: tinyint), _col3 (type: timestamp), _col1 (type: float), _col2 (type: string), (- _col0) (type: tinyint), _col5 (type: tinyint), ((- _col0) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col0) + _col5))) (type: double), (- _col6) (type: double), (79.553 * UDFToDouble(_col1)) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (UDFToDouble(((- _col0) + _col5)) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * UDFToDouble(((- _col0) + _col5))) / UDFToDouble(_col0)) (type: double), _col10 (type: tinyint)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
             Statistics: Num rows: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE
             File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorization_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_15.q.out b/ql/src/test/results/clientpositive/vectorization_15.q.out
index da0e8e0..be06613 100644
--- a/ql/src/test/results/clientpositive/vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_15.q.out
@@ -80,31 +80,31 @@ STAGE PLANS:
               predicate: ((cstring2 like '%ss%') or (cstring1 like '10%') or ((cint >= -75) and (UDFToShort(ctinyint) = csmallint) and (cdouble >= -3728.0))) (type: boolean)
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: cfloat (type: float), cboolean1 (type: boolean), cdouble (type: double), cstring1 (type: string), ctinyint (type: tinyint), cint (type: int), ctimestamp1 (type: timestamp)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                expressions: ctinyint (type: tinyint), cint (type: int), cfloat (type: float), cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
+                outputColumnNames: ctinyint, cint, cfloat, cdouble, cstring1, ctimestamp1, cboolean1
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: stddev_samp(_col0), min(_col2), stddev_samp(_col4), var_pop(_col4), var_samp(_col5), stddev_pop(_col5)
-                  keys: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
+                  aggregations: stddev_samp(cfloat), min(cdouble), stddev_samp(ctinyint), var_pop(ctinyint), var_samp(cint), stddev_pop(cint)
+                  keys: ctinyint (type: tinyint), cint (type: int), cfloat (type: float), cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp), cboolean1 (type: boolean)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
-                    key expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
+                    key expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: float), _col3 (type: double), _col4 (type: string), _col5 (type: timestamp), _col6 (type: boolean)
                     sort order: +++++++
-                    Map-reduce partition columns: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
+                    Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: int), _col2 (type: float), _col3 (type: double), _col4 (type: string), _col5 (type: timestamp), _col6 (type: boolean)
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: double), _col9 (type: struct<count:bigint,sum:double,variance:double>), _col10 (type: struct<count:bigint,sum:double,variance:double>), _col11 (type: struct<count:bigint,sum:double,variance:double>), _col12 (type: struct<count:bigint,sum:double,variance:double>)
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           aggregations: stddev_samp(VALUE._col0), min(VALUE._col1), stddev_samp(VALUE._col2), var_pop(VALUE._col3), var_samp(VALUE._col4), stddev_pop(VALUE._col5)
-          keys: KEY._col0 (type: float), KEY._col1 (type: boolean), KEY._col2 (type: double), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int), KEY._col6 (type: timestamp)
+          keys: KEY._col0 (type: tinyint), KEY._col1 (type: int), KEY._col2 (type: float), KEY._col3 (type: double), KEY._col4 (type: string), KEY._col5 (type: timestamp), KEY._col6 (type: boolean)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp), _col7 (type: double), (-26.28 - UDFToDouble(_col5)) (type: double), _col8 (type: double), (_col2 * 79.553) (type: double), (33.0 % _col0) (type: float), _col9 (type: double), _col10 (type: double), (-23.0 % _col2) (type: double), (- _col4) (type: tinyint), _col11 (type: double), (UDFToFloat(_col5) - _col0) (type: float), (-23 % UDFToInteger(_col4)) (type: int), (- (-26.28 - UDFToDouble(_col5))) (type: double), _col12 (type: double)
+            expressions: _col2 (type: float), _col6 (type: boolean), _col3 (type: double), _col4 (type: string), _col0 (type: tinyint), _col1 (type: int), _col5 (type: timestamp), _col7 (type: double), (-26.28 - UDFToDouble(_col1)) (type: double), _col8 (type: double), (_col3 * 79.553) (type: double), (33.0 % _col2) (type: float), _col9 (type: double), _col10 (type: double), (-23.0 % _col3) (type: double), (- _col0) (type: tinyint), _col11 (type: double), (UDFToFloat(_col1) - _col2) (type: float), (-23 % UDFToInteger(_col0)) (type: int), (- (-26.28 - UDFToDouble(_col1))) (type: double), _col12 (type: double)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
             Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
             File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorization_16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_16.q.out b/ql/src/test/results/clientpositive/vectorization_16.q.out
index 6ae3b4e..6ee9957 100644
--- a/ql/src/test/results/clientpositive/vectorization_16.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_16.q.out
@@ -57,31 +57,31 @@ STAGE PLANS:
               predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389) or (cstring1 < 'a'))) (type: boolean)
               Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp)
-                outputColumnNames: _col0, _col1, _col2
+                expressions: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
+                outputColumnNames: cdouble, cstring1, ctimestamp1
                 Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: count(_col1), stddev_samp(_col1), min(_col1)
-                  keys: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                  aggregations: count(cdouble), stddev_samp(cdouble), min(cdouble)
+                  keys: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                   Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                    key expressions: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                     sort order: +++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                    Map-reduce partition columns: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                     Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col3 (type: bigint), _col4 (type: struct<count:bigint,sum:double,variance:double>), _col5 (type: double)
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0), stddev_samp(VALUE._col1), min(VALUE._col2)
-          keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
+          keys: KEY._col0 (type: double), KEY._col1 (type: string), KEY._col2 (type: timestamp)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
           Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639) (type: double), (- (_col1 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col1) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
+            expressions: _col1 (type: string), _col0 (type: double), _col2 (type: timestamp), (_col0 - 9763215.5639) (type: double), (- (_col0 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col0) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
             Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
             File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorization_9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_9.q.out b/ql/src/test/results/clientpositive/vectorization_9.q.out
index 6ae3b4e..6ee9957 100644
--- a/ql/src/test/results/clientpositive/vectorization_9.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_9.q.out
@@ -57,31 +57,31 @@ STAGE PLANS:
               predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389) or (cstring1 < 'a'))) (type: boolean)
               Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp)
-                outputColumnNames: _col0, _col1, _col2
+                expressions: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
+                outputColumnNames: cdouble, cstring1, ctimestamp1
                 Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: count(_col1), stddev_samp(_col1), min(_col1)
-                  keys: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                  aggregations: count(cdouble), stddev_samp(cdouble), min(cdouble)
+                  keys: cdouble (type: double), cstring1 (type: string), ctimestamp1 (type: timestamp)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                   Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                    key expressions: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                     sort order: +++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
+                    Map-reduce partition columns: _col0 (type: double), _col1 (type: string), _col2 (type: timestamp)
                     Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col3 (type: bigint), _col4 (type: struct<count:bigint,sum:double,variance:double>), _col5 (type: double)
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0), stddev_samp(VALUE._col1), min(VALUE._col2)
-          keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
+          keys: KEY._col0 (type: double), KEY._col1 (type: string), KEY._col2 (type: timestamp)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
           Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639) (type: double), (- (_col1 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col1) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
+            expressions: _col1 (type: string), _col0 (type: double), _col2 (type: timestamp), (_col0 - 9763215.5639) (type: double), (- (_col0 - 9763215.5639)) (type: double), _col3 (type: bigint), _col4 (type: double), (- _col4) (type: double), (_col4 * UDFToDouble(_col3)) (type: double), _col5 (type: double), (9763215.5639 / _col0) (type: double), (UDFToDouble(_col3) / -1.389) (type: double), _col4 (type: double)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
             Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
             File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorization_limit.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_limit.q.out b/ql/src/test/results/clientpositive/vectorization_limit.q.out
index 7691a4d..9ff888c 100644
--- a/ql/src/test/results/clientpositive/vectorization_limit.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_limit.q.out
@@ -250,10 +250,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctinyint (type: tinyint)
-              outputColumnNames: _col0
+              outputColumnNames: ctinyint
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: tinyint)
+                keys: ctinyint (type: tinyint)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -334,11 +334,11 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctinyint (type: tinyint), cdouble (type: double)
-              outputColumnNames: _col0, _col1
+              outputColumnNames: ctinyint, cdouble
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(DISTINCT _col1)
-                keys: _col0 (type: tinyint), _col1 (type: double)
+                aggregations: count(DISTINCT cdouble)
+                keys: ctinyint (type: tinyint), cdouble (type: double)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -452,11 +452,11 @@ STAGE PLANS:
               Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: cdouble (type: double), ctinyint (type: tinyint)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: cdouble, ctinyint
                 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: sum(_col1)
-                  keys: _col0 (type: double)
+                  aggregations: sum(ctinyint)
+                  keys: cdouble (type: double)
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorization_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_pushdown.q.out b/ql/src/test/results/clientpositive/vectorization_pushdown.q.out
index 7205376..3aff6a3 100644
--- a/ql/src/test/results/clientpositive/vectorization_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_pushdown.q.out
@@ -20,10 +20,10 @@ STAGE PLANS:
               Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: cbigint (type: bigint)
-                outputColumnNames: _col0
+                outputColumnNames: cbigint
                 Statistics: Num rows: 4096 Data size: 880654 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: avg(_col0)
+                  aggregations: avg(cbigint)
                   mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorization_short_regress.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_short_regress.q.out b/ql/src/test/results/clientpositive/vectorization_short_regress.q.out
index 728f628..570e649 100644
--- a/ql/src/test/results/clientpositive/vectorization_short_regress.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_short_regress.q.out
@@ -151,10 +151,10 @@ STAGE PLANS:
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: cint (type: int), cdouble (type: double), csmallint (type: smallint), cfloat (type: float), ctinyint (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                outputColumnNames: cint, cdouble, csmallint, cfloat, ctinyint
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: avg(_col0), sum(_col1), stddev_pop(_col0), stddev_samp(_col2), var_samp(_col0), avg(_col3), stddev_samp(_col0), min(_col4), count(_col2)
+                  aggregations: avg(cint), sum(cdouble), stddev_pop(cint), stddev_samp(csmallint), var_samp(cint), avg(cfloat), stddev_samp(cint), min(ctinyint), count(csmallint)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                   Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
@@ -357,10 +357,10 @@ STAGE PLANS:
               Statistics: Num rows: 6826 Data size: 1467614 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: cint (type: int), cbigint (type: bigint), csmallint (type: smallint), cdouble (type: double), ctinyint (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                outputColumnNames: cint, cbigint, csmallint, cdouble, ctinyint
                 Statistics: Num rows: 6826 Data size: 1467614 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: max(_col0), var_pop(_col1), stddev_pop(_col2), max(_col3), avg(_col4), min(_col0), min(_col3), stddev_samp(_col2), var_samp(_col0)
+                  aggregations: max(cint), var_pop(cbigint), stddev_pop(csmallint), max(cdouble), avg(ctinyint), min(cint), min(cdouble), stddev_samp(csmallint), var_samp(cint)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -554,10 +554,10 @@ STAGE PLANS:
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: cbigint (type: bigint), ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cdouble (type: double)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                outputColumnNames: cbigint, ctinyint, csmallint, cint, cdouble
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: var_pop(_col0), count(), max(_col1), stddev_pop(_col2), max(_col3), stddev_samp(_col4), count(_col1), avg(_col1)
+                  aggregations: var_pop(cbigint), count(), max(ctinyint), stddev_pop(csmallint), max(cint), stddev_samp(cdouble), count(ctinyint), avg(ctinyint)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -730,10 +730,10 @@ STAGE PLANS:
               Statistics: Num rows: 8874 Data size: 1907941 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: ctinyint (type: tinyint), cbigint (type: bigint), cint (type: int), cfloat (type: float)
-                outputColumnNames: _col0, _col1, _col2, _col3
+                outputColumnNames: ctinyint, cbigint, cint, cfloat
                 Statistics: Num rows: 8874 Data size: 1907941 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: avg(_col0), max(_col1), stddev_samp(_col2), var_pop(_col2), var_pop(_col1), max(_col3)
+                  aggregations: avg(ctinyint), max(cbigint), stddev_samp(cint), var_pop(cint), var_pop(cbigint), max(cfloat)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                   Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -1825,11 +1825,11 @@ STAGE PLANS:
               Statistics: Num rows: 2503 Data size: 538153 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: csmallint (type: smallint), cbigint (type: bigint), ctinyint (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2
+                outputColumnNames: csmallint, cbigint, ctinyint
                 Statistics: Num rows: 2503 Data size: 538153 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: stddev_samp(_col0), sum(_col1), var_pop(_col2), count()
-                  keys: _col0 (type: smallint)
+                  aggregations: stddev_samp(csmallint), sum(cbigint), var_pop(ctinyint), count()
+                  keys: csmallint (type: smallint)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4
                   Statistics: Num rows: 2503 Data size: 538153 Basic stats: COMPLETE Column stats: NONE
@@ -2035,11 +2035,11 @@ STAGE PLANS:
               Statistics: Num rows: 2654 Data size: 570619 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: cdouble (type: double), cfloat (type: float)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: cdouble, cfloat
                 Statistics: Num rows: 2654 Data size: 570619 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: var_samp(_col0), count(_col1), sum(_col1), var_pop(_col0), stddev_pop(_col0), sum(_col0)
-                  keys: _col0 (type: double)
+                  aggregations: var_samp(cdouble), count(cfloat), sum(cfloat), var_pop(cdouble), stddev_pop(cdouble), sum(cdouble)
+                  keys: cdouble (type: double)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 2654 Data size: 570619 Basic stats: COMPLETE Column stats: NONE
@@ -2292,31 +2292,31 @@ STAGE PLANS:
               predicate: ((UDFToDouble(ctimestamp1) <> 0.0) and (((-257 <> UDFToInteger(ctinyint)) and cboolean2 is not null and cstring1 regexp '.*ss' and (-3.0 < UDFToDouble(ctimestamp1))) or (UDFToDouble(ctimestamp2) = -5.0) or ((UDFToDouble(ctimestamp1) < 0.0) and (cstring2 like '%b%')) or (cdouble = UDFToDouble(cint)) or (cboolean1 is null and (cfloat < UDFToFloat(cint))))) (type: boolean)
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: ctimestamp1 (type: timestamp), cstring1 (type: string), cint (type: int), csmallint (type: smallint), ctinyint (type: tinyint), cfloat (type: float), cdouble (type: double)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                expressions: cstring1 (type: string), ctimestamp1 (type: timestamp), cint (type: int), csmallint (type: smallint), ctinyint (type: tinyint), cfloat (type: float), cdouble (type: double)
+                outputColumnNames: cstring1, ctimestamp1, cint, csmallint, ctinyint, cfloat, cdouble
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: stddev_pop(_col2), avg(_col3), count(), min(_col4), var_samp(_col3), var_pop(_col5), avg(_col2), var_samp(_col5), avg(_col5), min(_col6), var_pop(_col3), stddev_pop(_col4), sum(_col2)
-                  keys: _col0 (type: timestamp), _col1 (type: string)
+                  aggregations: stddev_pop(cint), avg(csmallint), count(), min(ctinyint), var_samp(csmallint), var_pop(cfloat), avg(cint), var_samp(cfloat), avg(cfloat), min(cdouble), var_pop(csmallint), stddev_pop(ctinyint), sum(cint)
+                  keys: cstring1 (type: string), ctimestamp1 (type: timestamp)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
-                    key expressions: _col0 (type: timestamp), _col1 (type: string)
+                    key expressions: _col0 (type: string), _col1 (type: timestamp)
                     sort order: ++
-                    Map-reduce partition columns: _col0 (type: timestamp), _col1 (type: string)
+                    Map-reduce partition columns: _col0 (type: string), _col1 (type: timestamp)
                     Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col2 (type: struct<count:bigint,sum:double,variance:double>), _col3 (type: struct<count:bigint,sum:double,input:smallint>), _col4 (type: bigint), _col5 (type: tinyint), _col6 (type: struct<count:bigint,sum:double,variance:double>), _col7 (type: struct<count:bigint,sum:double,variance:double>), _col8 (type: struct<count:bigint,sum:double,input:int>), _col9 (type: struct<count:bigint,sum:double,variance:double>), _col10 (type: struct<count:bigint,sum:double,input:float>), _col11 (type: double), _col12 (type: struct<count:bigint,sum:double,variance:double>), _col13 (type: struct<count:bigint,sum:double,variance:double>), _col14 (type: bigint)
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           aggregations: stddev_pop(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), min(VALUE._col3), var_samp(VALUE._col4), var_pop(VALUE._col5), avg(VALUE._col6), var_samp(VALUE._col7), avg(VALUE._col8), min(VALUE._col9), var_pop(VALUE._col10), stddev_pop(VALUE._col11), sum(VALUE._col12)
-          keys: KEY._col0 (type: timestamp), KEY._col1 (type: string)
+          keys: KEY._col0 (type: string), KEY._col1 (type: timestamp)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
           Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: timestamp), _col1 (type: string), _col2 (type: double), (_col2 * 10.175) (type: double), (- _col2) (type: double), _col3 (type: double), (- _col2) (type: double), (-26.28 - _col2) (type: double), _col4 (type: bigint), (- _col4) (type: bigint), ((-26.28 - _col2) * (- _col2)) (type: double), _col5 (type: tinyint), (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4))) (type: double), (- (_col2 * 10.175)) (type: double), _col6 (type: double), (_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- _col2)) (type: double), (UDFToDouble((- _col4)) / _col2) (type: double), _col7 (type: double), (10.175 / _col3) (type: double), _col8 (type: double), _col9 (type: double), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- (_col2 * 10.175))) (type: double), _col10 (type: double), (((_col6 + (((-26.28 - _col2) * (- _col2
 )) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) * 10.175) (type: double), (10.175 % (10.175 / _col3)) (type: double), (- _col5) (type: tinyint), _col11 (type: double), _col12 (type: double), (- ((-26.28 - _col2) * (- _col2))) (type: double), ((- _col2) % _col10) (type: double), (-26.28 / UDFToDouble((- _col5))) (type: double), _col13 (type: double), _col14 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) / _col7) (type: double), (- (- _col4)) (type: bigint), _col4 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) % -26.28) (type: double)
+            expressions: _col1 (type: timestamp), _col0 (type: string), _col2 (type: double), (_col2 * 10.175) (type: double), (- _col2) (type: double), _col3 (type: double), (- _col2) (type: double), (-26.28 - _col2) (type: double), _col4 (type: bigint), (- _col4) (type: bigint), ((-26.28 - _col2) * (- _col2)) (type: double), _col5 (type: tinyint), (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4))) (type: double), (- (_col2 * 10.175)) (type: double), _col6 (type: double), (_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- _col2)) (type: double), (UDFToDouble((- _col4)) / _col2) (type: double), _col7 (type: double), (10.175 / _col3) (type: double), _col8 (type: double), _col9 (type: double), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) (type: double), (- (- (_col2 * 10.175))) (type: double), _col10 (type: double), (((_col6 + (((-26.28 - _col2) * (- _col2
 )) * UDFToDouble((- _col4)))) - (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) * 10.175) (type: double), (10.175 % (10.175 / _col3)) (type: double), (- _col5) (type: tinyint), _col11 (type: double), _col12 (type: double), (- ((-26.28 - _col2) * (- _col2))) (type: double), ((- _col2) % _col10) (type: double), (-26.28 / UDFToDouble((- _col5))) (type: double), _col13 (type: double), _col14 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) / _col7) (type: double), (- (- _col4)) (type: bigint), _col4 (type: bigint), ((_col6 + (((-26.28 - _col2) * (- _col2)) * UDFToDouble((- _col4)))) % -26.28) (type: double)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38
             Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
@@ -2628,11 +2628,11 @@ STAGE PLANS:
               Statistics: Num rows: 4778 Data size: 1027287 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: cboolean1 (type: boolean), cfloat (type: float), cbigint (type: bigint), cint (type: int), cdouble (type: double), ctinyint (type: tinyint), csmallint (type: smallint)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                outputColumnNames: cboolean1, cfloat, cbigint, cint, cdouble, ctinyint, csmallint
                 Statistics: Num rows: 4778 Data size: 1027287 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  aggregations: max(_col1), sum(_col2), var_samp(_col3), avg(_col4), min(_col2), var_pop(_col2), sum(_col3), stddev_samp(_col5), stddev_pop(_col6), avg(_col3)
-                  keys: _col0 (type: boolean)
+                  aggregations: max(cfloat), sum(cbigint), var_samp(cint), avg(cdouble), min(cbigint), var_pop(cbigint), sum(cint), stddev_samp(ctinyint), stddev_pop(csmallint), avg(cint)
+                  keys: cboolean1 (type: boolean)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
                   Statistics: Num rows: 4778 Data size: 1027287 Basic stats: COMPLETE Column stats: NONE
@@ -2863,10 +2863,10 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Select Operator
               expressions: i (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: i
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(i)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3027,10 +3027,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctinyint (type: tinyint)
-              outputColumnNames: _col0
+              outputColumnNames: ctinyint
               Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(ctinyint)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3087,10 +3087,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: cint (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: cint
               Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(cint)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3147,10 +3147,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: cfloat (type: float)
-              outputColumnNames: _col0
+              outputColumnNames: cfloat
               Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(cfloat)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3207,10 +3207,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: cstring1 (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: cstring1
               Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(cstring1)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -3267,10 +3267,10 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: cboolean1 (type: boolean)
-              outputColumnNames: _col0
+              outputColumnNames: cboolean1
               Statistics: Num rows: 12288 Data size: 168 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(_col0)
+                aggregations: count(cboolean1)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorized_distinct_gby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_distinct_gby.q.out b/ql/src/test/results/clientpositive/vectorized_distinct_gby.q.out
index 2e041a3..b5c667f 100644
--- a/ql/src/test/results/clientpositive/vectorized_distinct_gby.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_distinct_gby.q.out
@@ -33,12 +33,12 @@ STAGE PLANS:
             Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: a (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: a
               Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(DISTINCT _col0), count(DISTINCT _col0)
+                aggregations: sum(DISTINCT a), count(DISTINCT a)
                 bucketGroup: true
-                keys: _col0 (type: int)
+                keys: a (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: NONE
@@ -93,11 +93,11 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: cint (type: int)
-              outputColumnNames: _col0
+              outputColumnNames: cint
               Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(DISTINCT _col0), count(DISTINCT _col0), avg(DISTINCT _col0), std(DISTINCT _col0)
-                keys: _col0 (type: int)
+                aggregations: sum(DISTINCT cint), count(DISTINCT cint), avg(DISTINCT cint), std(DISTINCT cint)
+                keys: cint (type: int)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorized_nested_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_nested_mapjoin.q.out b/ql/src/test/results/clientpositive/vectorized_nested_mapjoin.q.out
index 82e90da..6a9532e 100644
--- a/ql/src/test/results/clientpositive/vectorized_nested_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_nested_mapjoin.q.out
@@ -15,14 +15,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_0:$hdt$_0:v1 
+        $hdt$_0:$hdt$_0:v1 
           Fetch Operator
             limit: -1
-        $hdt$_0:$hdt$_1:v1 
+        $hdt$_1:v1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_0:$hdt$_0:v1 
+        $hdt$_0:$hdt$_0:v1 
           TableScan
             alias: v1
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -37,7 +37,7 @@ STAGE PLANS:
                   keys:
                     0 _col0 (type: tinyint)
                     1 _col0 (type: tinyint)
-        $hdt$_0:$hdt$_1:v1 
+        $hdt$_1:v1 
           TableScan
             alias: v1
             Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
@@ -86,19 +86,15 @@ STAGE PLANS:
                         1 _col0 (type: smallint)
                       outputColumnNames: _col1
                       Statistics: Num rows: 7433 Data size: 1598388 Basic stats: COMPLETE Column stats: NONE
-                      Select Operator
-                        expressions: _col1 (type: double)
+                      Group By Operator
+                        aggregations: sum(_col1)
+                        mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 7433 Data size: 1598388 Basic stats: COMPLETE Column stats: NONE
-                        Group By Operator
-                          aggregations: sum(_col0)
-                          mode: hash
-                          outputColumnNames: _col0
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                          Reduce Output Operator
-                            sort order: 
-                            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                            value expressions: _col0 (type: double)
+                          value expressions: _col0 (type: double)
       Local Work:
         Map Reduce Local Work
       Execution mode: vectorized

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorized_parquet.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_parquet.q.out b/ql/src/test/results/clientpositive/vectorized_parquet.q.out
index 7b2dc6d..50e210c 100644
--- a/ql/src/test/results/clientpositive/vectorized_parquet.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_parquet.q.out
@@ -146,11 +146,11 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 73728 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctinyint (type: tinyint), cint (type: int), csmallint (type: smallint), cstring1 (type: string), cfloat (type: float), cdouble (type: double)
-              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+              outputColumnNames: ctinyint, cint, csmallint, cstring1, cfloat, cdouble
               Statistics: Num rows: 12288 Data size: 73728 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: max(_col1), min(_col2), count(_col3), avg(_col4), stddev_pop(_col5)
-                keys: _col0 (type: tinyint)
+                aggregations: max(cint), min(csmallint), count(cstring1), avg(cfloat), stddev_pop(cdouble)
+                keys: ctinyint (type: tinyint)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                 Statistics: Num rows: 12288 Data size: 73728 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorized_parquet_types.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_parquet_types.q.out b/ql/src/test/results/clientpositive/vectorized_parquet_types.q.out
index 8d1bddc..a3bd8aa 100644
--- a/ql/src/test/results/clientpositive/vectorized_parquet_types.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_parquet_types.q.out
@@ -291,11 +291,11 @@ STAGE PLANS:
             Statistics: Num rows: 22 Data size: 242 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctinyint (type: tinyint), cint (type: int), csmallint (type: smallint), cstring1 (type: string), cfloat (type: float), cdouble (type: double), cdecimal (type: decimal(4,2))
-              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+              outputColumnNames: ctinyint, cint, csmallint, cstring1, cfloat, cdouble, cdecimal
               Statistics: Num rows: 22 Data size: 242 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: max(_col1), min(_col2), count(_col3), avg(_col4), stddev_pop(_col5), max(_col6)
-                keys: _col0 (type: tinyint)
+                aggregations: max(cint), min(csmallint), count(cstring1), avg(cfloat), stddev_pop(cdouble), max(cdecimal)
+                keys: ctinyint (type: tinyint)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 22 Data size: 242 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/1cce5f00/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out b/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
index f66903b..830eb92 100644
--- a/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
@@ -611,10 +611,10 @@ STAGE PLANS:
             Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctimestamp1 (type: timestamp)
-              outputColumnNames: _col0
+              outputColumnNames: ctimestamp1
               Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: min(_col0), max(_col0), count(_col0), count()
+                aggregations: min(ctimestamp1), max(ctimestamp1), count(ctimestamp1), count()
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: NONE
@@ -685,10 +685,10 @@ STAGE PLANS:
             Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctimestamp1 (type: timestamp)
-              outputColumnNames: _col0
+              outputColumnNames: ctimestamp1
               Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: sum(_col0)
+                aggregations: sum(ctimestamp1)
                 mode: hash
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -768,10 +768,10 @@ STAGE PLANS:
             Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ctimestamp1 (type: timestamp)
-              outputColumnNames: _col0
+              outputColumnNames: ctimestamp1
               Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: avg(_col0), variance(_col0), var_pop(_col0), var_samp(_col0), std(_col0), stddev(_col0), stddev_pop(_col0), stddev_samp(_col0)
+                aggregations: avg(ctimestamp1), variance(ctimestamp1), var_pop(ctimestamp1), var_samp(ctimestamp1), std(ctimestamp1), stddev(ctimestamp1), stddev_pop(ctimestamp1), stddev_samp(ctimestamp1)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE


[22/50] [abbrv] hive git commit: HIVE-11792: User explain in tez does not preserve ordering (Prasanth Jayachandran reviewed by Pengcheng Xiong)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_2.q.out b/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
index 566b451..8156789 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
@@ -192,13 +192,13 @@ Stage-0
          File Output Operator [FS_18]
             compressed:false
             Statistics:Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_17]
                outputColumnNames:["_col0","_col1","_col2"]
                Statistics:Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
                Merge Join Operator [MERGEJOIN_28]
                |  condition map:[{"":"Inner Join 0 to 1"}]
-               |  keys:{"1":"_col0 (type: string)","0":"_col3 (type: string)"}
+               |  keys:{"0":"_col3 (type: string)","1":"_col0 (type: string)"}
                |  outputColumnNames:["_col0","_col3","_col6"]
                |  Statistics:Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
                |<-Map 5 [SIMPLE_EDGE]
@@ -226,7 +226,7 @@ Stage-0
                      value expressions:_col0 (type: string)
                      Merge Join Operator [MERGEJOIN_27]
                      |  condition map:[{"":"Inner Join 0 to 1"}]
-                     |  keys:{"1":"_col1 (type: string)","0":"_col0 (type: string)"}
+                     |  keys:{"0":"_col0 (type: string)","1":"_col1 (type: string)"}
                      |  outputColumnNames:["_col0","_col3"]
                      |  Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
                      |<-Map 1 [SIMPLE_EDGE]
@@ -338,7 +338,7 @@ Stage-0
          File Output Operator [FS_69]
             compressed:false
             Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Limit [LIM_68]
                Number of rows:100
                Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
@@ -373,7 +373,7 @@ Stage-0
                                  Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
                                  Merge Join Operator [MERGEJOIN_111]
                                  |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |  keys:{"1":"_col15 (type: string), _col17 (type: string)","0":"_col1 (type: string), _col3 (type: string)"}
+                                 |  keys:{"0":"_col1 (type: string), _col3 (type: string)","1":"_col15 (type: string), _col17 (type: string)"}
                                  |  outputColumnNames:["_col2","_col3","_col12","_col13","_col20","_col21"]
                                  |  Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
                                  |<-Reducer 11 [SIMPLE_EDGE]
@@ -388,7 +388,7 @@ Stage-0
                                  |        Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
                                  |        Merge Join Operator [MERGEJOIN_110]
                                  |        |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |        |  keys:{"1":"_col2 (type: string), _col4 (type: string)","0":"_col4 (type: string), _col6 (type: string)"}
+                                 |        |  keys:{"0":"_col4 (type: string), _col6 (type: string)","1":"_col2 (type: string), _col4 (type: string)"}
                                  |        |  outputColumnNames:["_col2","_col3","_col14","_col15","_col17"]
                                  |        |  Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
                                  |        |<-Reducer 10 [SIMPLE_EDGE]
@@ -400,7 +400,7 @@ Stage-0
                                  |        |     value expressions:_col2 (type: string), _col3 (type: string)
                                  |        |     Merge Join Operator [MERGEJOIN_108]
                                  |        |     |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |        |     |  keys:{"1":"_col1 (type: string)","0":"_col3 (type: string)"}
+                                 |        |     |  keys:{"0":"_col3 (type: string)","1":"_col1 (type: string)"}
                                  |        |     |  outputColumnNames:["_col2","_col3","_col4","_col6"]
                                  |        |     |  Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
                                  |        |     |<-Map 14 [SIMPLE_EDGE]
@@ -427,7 +427,7 @@ Stage-0
                                  |        |           value expressions:_col2 (type: string), _col4 (type: string), _col6 (type: string)
                                  |        |           Merge Join Operator [MERGEJOIN_107]
                                  |        |           |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |        |           |  keys:{"1":"_col0 (type: string)","0":"_col2 (type: string)"}
+                                 |        |           |  keys:{"0":"_col2 (type: string)","1":"_col0 (type: string)"}
                                  |        |           |  outputColumnNames:["_col2","_col3","_col4","_col6"]
                                  |        |           |  Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                                  |        |           |<-Map 13 [SIMPLE_EDGE]
@@ -454,7 +454,7 @@ Stage-0
                                  |        |                 value expressions:_col3 (type: string), _col4 (type: string), _col6 (type: string)
                                  |        |                 Merge Join Operator [MERGEJOIN_106]
                                  |        |                 |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |        |                 |  keys:{"1":"_col3 (type: string)","0":"_col1 (type: string)"}
+                                 |        |                 |  keys:{"0":"_col1 (type: string)","1":"_col3 (type: string)"}
                                  |        |                 |  outputColumnNames:["_col2","_col3","_col4","_col6"]
                                  |        |                 |  Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                                  |        |                 |<-Map 12 [SIMPLE_EDGE]
@@ -497,7 +497,7 @@ Stage-0
                                  |              value expressions:_col3 (type: string), _col5 (type: string)
                                  |              Merge Join Operator [MERGEJOIN_109]
                                  |              |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |              |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                 |              |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                  |              |  outputColumnNames:["_col2","_col3","_col4","_col5"]
                                  |              |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
                                  |              |<-Map 15 [SIMPLE_EDGE]
@@ -540,7 +540,7 @@ Stage-0
                                        value expressions:_col2 (type: string)
                                        Merge Join Operator [MERGEJOIN_105]
                                        |  condition map:[{"":"Inner Join 0 to 1"}]
-                                       |  keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+                                       |  keys:{"0":"_col0 (type: string)","1":"_col0 (type: string)"}
                                        |  outputColumnNames:["_col1","_col2","_col3"]
                                        |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
                                        |<-Map 1 [SIMPLE_EDGE]
@@ -616,13 +616,13 @@ Stage-0
          File Output Operator [FS_59]
             compressed:false
             Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_57]
             |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
             |  outputColumnNames:["_col0","_col1"]
             |  Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
             |<-Union 6 [SIMPLE_EDGE]
-               |<-Reducer 5 [CONTAINS]
+               |<-Reducer 15 [CONTAINS]
                |  Reduce Output Operator [RS_56]
                |     key expressions:_col0 (type: string), _col1 (type: string)
                |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
@@ -632,110 +632,110 @@ Stage-0
                |        keys:_col0 (type: string), _col1 (type: string)
                |        outputColumnNames:["_col0","_col1"]
                |        Statistics:Num rows: 550 Data size: 5842 Basic stats: COMPLETE Column stats: NONE
-               |        Select Operator [SEL_25]
+               |        Select Operator [SEL_51]
                |           outputColumnNames:["_col0","_col1"]
                |           Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-               |           Merge Join Operator [MERGEJOIN_83]
+               |           Merge Join Operator [MERGEJOIN_85]
                |           |  condition map:[{"":"Inner Join 0 to 1"}]
-               |           |  keys:{"1":"_col0 (type: string)","0":"_col2 (type: string)"}
+               |           |  keys:{"0":"_col2 (type: string)","1":"_col0 (type: string)"}
                |           |  outputColumnNames:["_col1","_col2"]
                |           |  Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-               |           |<-Map 10 [SIMPLE_EDGE]
-               |           |  Reduce Output Operator [RS_23]
+               |           |<-Map 18 [SIMPLE_EDGE]
+               |           |  Reduce Output Operator [RS_49]
                |           |     key expressions:_col0 (type: string)
                |           |     Map-reduce partition columns:_col0 (type: string)
                |           |     sort order:+
                |           |     Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-               |           |     Select Operator [SEL_14]
+               |           |     Select Operator [SEL_40]
                |           |        outputColumnNames:["_col0"]
                |           |        Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-               |           |        Filter Operator [FIL_77]
+               |           |        Filter Operator [FIL_81]
                |           |           predicate:key is not null (type: boolean)
                |           |           Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-               |           |           TableScan [TS_13]
+               |           |           TableScan [TS_39]
                |           |              alias:y
                |           |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-               |           |<-Reducer 4 [SIMPLE_EDGE]
-               |              Reduce Output Operator [RS_21]
+               |           |<-Reducer 14 [SIMPLE_EDGE]
+               |              Reduce Output Operator [RS_47]
                |                 key expressions:_col2 (type: string)
                |                 Map-reduce partition columns:_col2 (type: string)
                |                 sort order:+
                |                 Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
                |                 value expressions:_col1 (type: string)
-               |                 Merge Join Operator [MERGEJOIN_82]
+               |                 Merge Join Operator [MERGEJOIN_84]
                |                 |  condition map:[{"":"Inner Join 0 to 1"}]
-               |                 |  keys:{"1":"_col1 (type: string)","0":"_col1 (type: string)"}
+               |                 |  keys:{"0":"_col1 (type: string)","1":"_col1 (type: string)"}
                |                 |  outputColumnNames:["_col1","_col2"]
                |                 |  Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
-               |                 |<-Map 9 [SIMPLE_EDGE]
-               |                 |  Reduce Output Operator [RS_18]
+               |                 |<-Map 17 [SIMPLE_EDGE]
+               |                 |  Reduce Output Operator [RS_44]
                |                 |     key expressions:_col1 (type: string)
                |                 |     Map-reduce partition columns:_col1 (type: string)
                |                 |     sort order:+
                |                 |     Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
                |                 |     value expressions:_col0 (type: string)
-               |                 |     Select Operator [SEL_12]
+               |                 |     Select Operator [SEL_38]
                |                 |        outputColumnNames:["_col0","_col1"]
                |                 |        Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
-               |                 |        Filter Operator [FIL_76]
+               |                 |        Filter Operator [FIL_80]
                |                 |           predicate:(value is not null and key is not null) (type: boolean)
                |                 |           Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
-               |                 |           TableScan [TS_11]
+               |                 |           TableScan [TS_37]
                |                 |              alias:x
                |                 |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-               |                 |<-Reducer 3 [SIMPLE_EDGE]
-               |                    Reduce Output Operator [RS_16]
+               |                 |<-Reducer 13 [SIMPLE_EDGE]
+               |                    Reduce Output Operator [RS_42]
                |                       key expressions:_col1 (type: string)
                |                       Map-reduce partition columns:_col1 (type: string)
                |                       sort order:+
                |                       Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-               |                       Select Operator [SEL_10]
+               |                       Select Operator [SEL_36]
                |                          outputColumnNames:["_col1"]
                |                          Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-               |                          Group By Operator [GBY_9]
+               |                          Group By Operator [GBY_35]
                |                          |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
                |                          |  outputColumnNames:["_col0","_col1"]
                |                          |  Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-               |                          |<-Union 2 [SIMPLE_EDGE]
-               |                             |<-Map 1 [CONTAINS]
-               |                             |  Reduce Output Operator [RS_8]
+               |                          |<-Union 12 [SIMPLE_EDGE]
+               |                             |<-Map 11 [CONTAINS]
+               |                             |  Reduce Output Operator [RS_34]
                |                             |     key expressions:_col0 (type: string), _col1 (type: string)
                |                             |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
                |                             |     sort order:++
                |                             |     Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-               |                             |     Group By Operator [GBY_7]
+               |                             |     Group By Operator [GBY_33]
                |                             |        keys:_col0 (type: string), _col1 (type: string)
                |                             |        outputColumnNames:["_col0","_col1"]
                |                             |        Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-               |                             |        Select Operator [SEL_1]
+               |                             |        Select Operator [SEL_27]
                |                             |           outputColumnNames:["_col0","_col1"]
                |                             |           Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-               |                             |           Filter Operator [FIL_74]
+               |                             |           Filter Operator [FIL_78]
                |                             |              predicate:value is not null (type: boolean)
                |                             |              Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-               |                             |              TableScan [TS_0]
+               |                             |              TableScan [TS_26]
                |                             |                 alias:x
                |                             |                 Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-               |                             |<-Map 8 [CONTAINS]
-               |                                Reduce Output Operator [RS_8]
+               |                             |<-Map 16 [CONTAINS]
+               |                                Reduce Output Operator [RS_34]
                |                                   key expressions:_col0 (type: string), _col1 (type: string)
                |                                   Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
                |                                   sort order:++
                |                                   Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-               |                                   Group By Operator [GBY_7]
+               |                                   Group By Operator [GBY_33]
                |                                      keys:_col0 (type: string), _col1 (type: string)
                |                                      outputColumnNames:["_col0","_col1"]
                |                                      Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-               |                                      Select Operator [SEL_3]
+               |                                      Select Operator [SEL_29]
                |                                         outputColumnNames:["_col0","_col1"]
                |                                         Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-               |                                         Filter Operator [FIL_75]
+               |                                         Filter Operator [FIL_79]
                |                                            predicate:value is not null (type: boolean)
                |                                            Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-               |                                            TableScan [TS_2]
+               |                                            TableScan [TS_28]
                |                                               alias:y
                |                                               Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-               |<-Reducer 15 [CONTAINS]
+               |<-Reducer 5 [CONTAINS]
                   Reduce Output Operator [RS_56]
                      key expressions:_col0 (type: string), _col1 (type: string)
                      Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
@@ -745,107 +745,107 @@ Stage-0
                         keys:_col0 (type: string), _col1 (type: string)
                         outputColumnNames:["_col0","_col1"]
                         Statistics:Num rows: 550 Data size: 5842 Basic stats: COMPLETE Column stats: NONE
-                        Select Operator [SEL_51]
+                        Select Operator [SEL_25]
                            outputColumnNames:["_col0","_col1"]
                            Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-                           Merge Join Operator [MERGEJOIN_85]
+                           Merge Join Operator [MERGEJOIN_83]
                            |  condition map:[{"":"Inner Join 0 to 1"}]
-                           |  keys:{"1":"_col0 (type: string)","0":"_col2 (type: string)"}
+                           |  keys:{"0":"_col2 (type: string)","1":"_col0 (type: string)"}
                            |  outputColumnNames:["_col1","_col2"]
                            |  Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-                           |<-Map 18 [SIMPLE_EDGE]
-                           |  Reduce Output Operator [RS_49]
+                           |<-Map 10 [SIMPLE_EDGE]
+                           |  Reduce Output Operator [RS_23]
                            |     key expressions:_col0 (type: string)
                            |     Map-reduce partition columns:_col0 (type: string)
                            |     sort order:+
                            |     Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                           |     Select Operator [SEL_40]
+                           |     Select Operator [SEL_14]
                            |        outputColumnNames:["_col0"]
                            |        Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                           |        Filter Operator [FIL_81]
+                           |        Filter Operator [FIL_77]
                            |           predicate:key is not null (type: boolean)
                            |           Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                           |           TableScan [TS_39]
+                           |           TableScan [TS_13]
                            |              alias:y
                            |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                           |<-Reducer 14 [SIMPLE_EDGE]
-                              Reduce Output Operator [RS_47]
+                           |<-Reducer 4 [SIMPLE_EDGE]
+                              Reduce Output Operator [RS_21]
                                  key expressions:_col2 (type: string)
                                  Map-reduce partition columns:_col2 (type: string)
                                  sort order:+
                                  Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
                                  value expressions:_col1 (type: string)
-                                 Merge Join Operator [MERGEJOIN_84]
+                                 Merge Join Operator [MERGEJOIN_82]
                                  |  condition map:[{"":"Inner Join 0 to 1"}]
-                                 |  keys:{"1":"_col1 (type: string)","0":"_col1 (type: string)"}
+                                 |  keys:{"0":"_col1 (type: string)","1":"_col1 (type: string)"}
                                  |  outputColumnNames:["_col1","_col2"]
                                  |  Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
-                                 |<-Map 17 [SIMPLE_EDGE]
-                                 |  Reduce Output Operator [RS_44]
+                                 |<-Map 9 [SIMPLE_EDGE]
+                                 |  Reduce Output Operator [RS_18]
                                  |     key expressions:_col1 (type: string)
                                  |     Map-reduce partition columns:_col1 (type: string)
                                  |     sort order:+
                                  |     Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
                                  |     value expressions:_col0 (type: string)
-                                 |     Select Operator [SEL_38]
+                                 |     Select Operator [SEL_12]
                                  |        outputColumnNames:["_col0","_col1"]
                                  |        Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
-                                 |        Filter Operator [FIL_80]
+                                 |        Filter Operator [FIL_76]
                                  |           predicate:(value is not null and key is not null) (type: boolean)
                                  |           Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
-                                 |           TableScan [TS_37]
+                                 |           TableScan [TS_11]
                                  |              alias:x
                                  |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                                 |<-Reducer 13 [SIMPLE_EDGE]
-                                    Reduce Output Operator [RS_42]
+                                 |<-Reducer 3 [SIMPLE_EDGE]
+                                    Reduce Output Operator [RS_16]
                                        key expressions:_col1 (type: string)
                                        Map-reduce partition columns:_col1 (type: string)
                                        sort order:+
                                        Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-                                       Select Operator [SEL_36]
+                                       Select Operator [SEL_10]
                                           outputColumnNames:["_col1"]
                                           Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-                                          Group By Operator [GBY_35]
+                                          Group By Operator [GBY_9]
                                           |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
                                           |  outputColumnNames:["_col0","_col1"]
                                           |  Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-                                          |<-Union 12 [SIMPLE_EDGE]
-                                             |<-Map 11 [CONTAINS]
-                                             |  Reduce Output Operator [RS_34]
+                                          |<-Union 2 [SIMPLE_EDGE]
+                                             |<-Map 1 [CONTAINS]
+                                             |  Reduce Output Operator [RS_8]
                                              |     key expressions:_col0 (type: string), _col1 (type: string)
                                              |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
                                              |     sort order:++
                                              |     Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                                             |     Group By Operator [GBY_33]
+                                             |     Group By Operator [GBY_7]
                                              |        keys:_col0 (type: string), _col1 (type: string)
                                              |        outputColumnNames:["_col0","_col1"]
                                              |        Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                                             |        Select Operator [SEL_27]
+                                             |        Select Operator [SEL_1]
                                              |           outputColumnNames:["_col0","_col1"]
                                              |           Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                                             |           Filter Operator [FIL_78]
+                                             |           Filter Operator [FIL_74]
                                              |              predicate:value is not null (type: boolean)
                                              |              Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                                             |              TableScan [TS_26]
+                                             |              TableScan [TS_0]
                                              |                 alias:x
                                              |                 Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                                             |<-Map 16 [CONTAINS]
-                                                Reduce Output Operator [RS_34]
+                                             |<-Map 8 [CONTAINS]
+                                                Reduce Output Operator [RS_8]
                                                    key expressions:_col0 (type: string), _col1 (type: string)
                                                    Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
                                                    sort order:++
                                                    Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                                                   Group By Operator [GBY_33]
+                                                   Group By Operator [GBY_7]
                                                       keys:_col0 (type: string), _col1 (type: string)
                                                       outputColumnNames:["_col0","_col1"]
                                                       Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                                                      Select Operator [SEL_29]
+                                                      Select Operator [SEL_3]
                                                          outputColumnNames:["_col0","_col1"]
                                                          Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                                                         Filter Operator [FIL_79]
+                                                         Filter Operator [FIL_75]
                                                             predicate:value is not null (type: boolean)
                                                             Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                                                            TableScan [TS_28]
+                                                            TableScan [TS_2]
                                                                alias:y
                                                                Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 
@@ -910,7 +910,7 @@ Stage-0
          File Output Operator [FS_119]
             compressed:false
             Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_117]
             |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
             |  outputColumnNames:["_col0","_col1"]
@@ -931,7 +931,7 @@ Stage-0
                |           Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
                |           Merge Join Operator [MERGEJOIN_167]
                |           |  condition map:[{"":"Inner Join 0 to 1"}]
-               |           |  keys:{"1":"_col0 (type: string)","0":"_col2 (type: string)"}
+               |           |  keys:{"0":"_col2 (type: string)","1":"_col0 (type: string)"}
                |           |  outputColumnNames:["_col2","_col5"]
                |           |  Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
                |           |<-Map 37 [SIMPLE_EDGE]
@@ -958,7 +958,7 @@ Stage-0
                |                 Statistics:Num rows: 242 Data size: 2565 Basic stats: COMPLETE Column stats: NONE
                |                 Merge Join Operator [MERGEJOIN_166]
                |                 |  condition map:[{"":"Inner Join 0 to 1"}]
-               |                 |  keys:{"1":"_col1 (type: string)","0":"_col1 (type: string)"}
+               |                 |  keys:{"0":"_col1 (type: string)","1":"_col1 (type: string)"}
                |                 |  outputColumnNames:["_col2"]
                |                 |  Statistics:Num rows: 242 Data size: 2565 Basic stats: COMPLETE Column stats: NONE
                |                 |<-Map 36 [SIMPLE_EDGE]
@@ -1112,7 +1112,7 @@ Stage-0
                         |  outputColumnNames:["_col0","_col1"]
                         |  Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
                         |<-Union 6 [SIMPLE_EDGE]
-                           |<-Reducer 5 [CONTAINS]
+                           |<-Reducer 19 [CONTAINS]
                            |  Reduce Output Operator [RS_65]
                            |     key expressions:_col0 (type: string), _col1 (type: string)
                            |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
@@ -1122,110 +1122,144 @@ Stage-0
                            |        keys:_col0 (type: string), _col1 (type: string)
                            |        outputColumnNames:["_col0","_col1"]
                            |        Statistics:Num rows: 550 Data size: 5842 Basic stats: COMPLETE Column stats: NONE
-                           |        Select Operator [SEL_25]
+                           |        Select Operator [SEL_60]
                            |           outputColumnNames:["_col0","_col1"]
                            |           Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-                           |           Merge Join Operator [MERGEJOIN_163]
+                           |           Merge Join Operator [MERGEJOIN_165]
                            |           |  condition map:[{"":"Inner Join 0 to 1"}]
-                           |           |  keys:{"1":"_col0 (type: string)","0":"_col2 (type: string)"}
+                           |           |  keys:{"0":"_col2 (type: string)","1":"_col0 (type: string)"}
                            |           |  outputColumnNames:["_col2","_col5"]
                            |           |  Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-                           |           |<-Map 12 [SIMPLE_EDGE]
-                           |           |  Reduce Output Operator [RS_23]
+                           |           |<-Map 23 [SIMPLE_EDGE]
+                           |           |  Reduce Output Operator [RS_58]
                            |           |     key expressions:_col0 (type: string)
                            |           |     Map-reduce partition columns:_col0 (type: string)
                            |           |     sort order:+
                            |           |     Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                            |           |     value expressions:_col1 (type: string)
-                           |           |     Select Operator [SEL_14]
+                           |           |     Select Operator [SEL_49]
                            |           |        outputColumnNames:["_col0","_col1"]
                            |           |        Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                           |           |        Filter Operator [FIL_150]
+                           |           |        Filter Operator [FIL_155]
                            |           |           predicate:key is not null (type: boolean)
                            |           |           Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                           |           |           TableScan [TS_13]
+                           |           |           TableScan [TS_48]
                            |           |              alias:y
                            |           |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                           |           |<-Reducer 4 [SIMPLE_EDGE]
-                           |              Reduce Output Operator [RS_21]
+                           |           |<-Reducer 18 [SIMPLE_EDGE]
+                           |              Reduce Output Operator [RS_56]
                            |                 key expressions:_col2 (type: string)
                            |                 Map-reduce partition columns:_col2 (type: string)
                            |                 sort order:+
-                           |                 Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
-                           |                 Merge Join Operator [MERGEJOIN_162]
+                           |                 Statistics:Num rows: 209 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
+                           |                 Merge Join Operator [MERGEJOIN_164]
                            |                 |  condition map:[{"":"Inner Join 0 to 1"}]
-                           |                 |  keys:{"1":"_col1 (type: string)","0":"_col1 (type: string)"}
+                           |                 |  keys:{"0":"_col1 (type: string)","1":"_col1 (type: string)"}
                            |                 |  outputColumnNames:["_col2"]
-                           |                 |  Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
-                           |                 |<-Map 11 [SIMPLE_EDGE]
-                           |                 |  Reduce Output Operator [RS_18]
+                           |                 |  Statistics:Num rows: 209 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
+                           |                 |<-Map 22 [SIMPLE_EDGE]
+                           |                 |  Reduce Output Operator [RS_53]
                            |                 |     key expressions:_col1 (type: string)
                            |                 |     Map-reduce partition columns:_col1 (type: string)
                            |                 |     sort order:+
                            |                 |     Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
                            |                 |     value expressions:_col0 (type: string)
-                           |                 |     Select Operator [SEL_12]
+                           |                 |     Select Operator [SEL_47]
                            |                 |        outputColumnNames:["_col0","_col1"]
                            |                 |        Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
-                           |                 |        Filter Operator [FIL_149]
+                           |                 |        Filter Operator [FIL_154]
                            |                 |           predicate:(value is not null and key is not null) (type: boolean)
                            |                 |           Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
-                           |                 |           TableScan [TS_11]
+                           |                 |           TableScan [TS_46]
                            |                 |              alias:x
                            |                 |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                           |                 |<-Reducer 3 [SIMPLE_EDGE]
-                           |                    Reduce Output Operator [RS_16]
+                           |                 |<-Reducer 17 [SIMPLE_EDGE]
+                           |                    Reduce Output Operator [RS_51]
                            |                       key expressions:_col1 (type: string)
                            |                       Map-reduce partition columns:_col1 (type: string)
                            |                       sort order:+
-                           |                       Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-                           |                       Select Operator [SEL_10]
+                           |                       Statistics:Num rows: 190 Data size: 2008 Basic stats: COMPLETE Column stats: NONE
+                           |                       Select Operator [SEL_45]
                            |                          outputColumnNames:["_col1"]
-                           |                          Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-                           |                          Group By Operator [GBY_9]
+                           |                          Statistics:Num rows: 190 Data size: 2008 Basic stats: COMPLETE Column stats: NONE
+                           |                          Group By Operator [GBY_44]
                            |                          |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
                            |                          |  outputColumnNames:["_col0","_col1"]
-                           |                          |  Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-                           |                          |<-Union 2 [SIMPLE_EDGE]
-                           |                             |<-Map 1 [CONTAINS]
-                           |                             |  Reduce Output Operator [RS_8]
+                           |                          |  Statistics:Num rows: 190 Data size: 2008 Basic stats: COMPLETE Column stats: NONE
+                           |                          |<-Union 16 [SIMPLE_EDGE]
+                           |                             |<-Map 21 [CONTAINS]
+                           |                             |  Reduce Output Operator [RS_43]
                            |                             |     key expressions:_col0 (type: string), _col1 (type: string)
                            |                             |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
                            |                             |     sort order:++
-                           |                             |     Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                           |                             |     Group By Operator [GBY_7]
+                           |                             |     Statistics:Num rows: 381 Data size: 4028 Basic stats: COMPLETE Column stats: NONE
+                           |                             |     Group By Operator [GBY_42]
                            |                             |        keys:_col0 (type: string), _col1 (type: string)
                            |                             |        outputColumnNames:["_col0","_col1"]
-                           |                             |        Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                           |                             |        Select Operator [SEL_1]
+                           |                             |        Statistics:Num rows: 381 Data size: 4028 Basic stats: COMPLETE Column stats: NONE
+                           |                             |        Select Operator [SEL_38]
                            |                             |           outputColumnNames:["_col0","_col1"]
-                           |                             |           Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                           |                             |           Filter Operator [FIL_147]
+                           |                             |           Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                           |                             |           Filter Operator [FIL_153]
                            |                             |              predicate:value is not null (type: boolean)
-                           |                             |              Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                           |                             |              TableScan [TS_0]
-                           |                             |                 alias:x
-                           |                             |                 Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                           |                             |<-Map 10 [CONTAINS]
-                           |                                Reduce Output Operator [RS_8]
+                           |                             |              Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                           |                             |              TableScan [TS_37]
+                           |                             |                 alias:y
+                           |                             |                 Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                           |                             |<-Reducer 15 [CONTAINS]
+                           |                                Reduce Output Operator [RS_43]
                            |                                   key expressions:_col0 (type: string), _col1 (type: string)
                            |                                   Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
                            |                                   sort order:++
-                           |                                   Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                           |                                   Group By Operator [GBY_7]
+                           |                                   Statistics:Num rows: 381 Data size: 4028 Basic stats: COMPLETE Column stats: NONE
+                           |                                   Group By Operator [GBY_42]
                            |                                      keys:_col0 (type: string), _col1 (type: string)
                            |                                      outputColumnNames:["_col0","_col1"]
-                           |                                      Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                           |                                      Select Operator [SEL_3]
-                           |                                         outputColumnNames:["_col0","_col1"]
-                           |                                         Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                           |                                         Filter Operator [FIL_148]
-                           |                                            predicate:value is not null (type: boolean)
-                           |                                            Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                           |                                            TableScan [TS_2]
-                           |                                               alias:y
-                           |                                               Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                           |<-Reducer 19 [CONTAINS]
+                           |                                      Statistics:Num rows: 381 Data size: 4028 Basic stats: COMPLETE Column stats: NONE
+                           |                                      Group By Operator [GBY_35]
+                           |                                      |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
+                           |                                      |  outputColumnNames:["_col0","_col1"]
+                           |                                      |  Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
+                           |                                      |<-Union 14 [SIMPLE_EDGE]
+                           |                                         |<-Map 13 [CONTAINS]
+                           |                                         |  Reduce Output Operator [RS_34]
+                           |                                         |     key expressions:_col0 (type: string), _col1 (type: string)
+                           |                                         |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
+                           |                                         |     sort order:++
+                           |                                         |     Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
+                           |                                         |     Group By Operator [GBY_33]
+                           |                                         |        keys:_col0 (type: string), _col1 (type: string)
+                           |                                         |        outputColumnNames:["_col0","_col1"]
+                           |                                         |        Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
+                           |                                         |        Select Operator [SEL_27]
+                           |                                         |           outputColumnNames:["_col0","_col1"]
+                           |                                         |           Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+                           |                                         |           Filter Operator [FIL_151]
+                           |                                         |              predicate:value is not null (type: boolean)
+                           |                                         |              Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+                           |                                         |              TableScan [TS_26]
+                           |                                         |                 alias:x
+                           |                                         |                 Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                           |                                         |<-Map 20 [CONTAINS]
+                           |                                            Reduce Output Operator [RS_34]
+                           |                                               key expressions:_col0 (type: string), _col1 (type: string)
+                           |                                               Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
+                           |                                               sort order:++
+                           |                                               Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
+                           |                                               Group By Operator [GBY_33]
+                           |                                                  keys:_col0 (type: string), _col1 (type: string)
+                           |                                                  outputColumnNames:["_col0","_col1"]
+                           |                                                  Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
+                           |                                                  Select Operator [SEL_29]
+                           |                                                     outputColumnNames:["_col0","_col1"]
+                           |                                                     Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                           |                                                     Filter Operator [FIL_152]
+                           |                                                        predicate:value is not null (type: boolean)
+                           |                                                        Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                           |                                                        TableScan [TS_28]
+                           |                                                           alias:y
+                           |                                                           Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                           |<-Reducer 5 [CONTAINS]
                               Reduce Output Operator [RS_65]
                                  key expressions:_col0 (type: string), _col1 (type: string)
                                  Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
@@ -1235,143 +1269,109 @@ Stage-0
                                     keys:_col0 (type: string), _col1 (type: string)
                                     outputColumnNames:["_col0","_col1"]
                                     Statistics:Num rows: 550 Data size: 5842 Basic stats: COMPLETE Column stats: NONE
-                                    Select Operator [SEL_60]
+                                    Select Operator [SEL_25]
                                        outputColumnNames:["_col0","_col1"]
                                        Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-                                       Merge Join Operator [MERGEJOIN_165]
+                                       Merge Join Operator [MERGEJOIN_163]
                                        |  condition map:[{"":"Inner Join 0 to 1"}]
-                                       |  keys:{"1":"_col0 (type: string)","0":"_col2 (type: string)"}
+                                       |  keys:{"0":"_col2 (type: string)","1":"_col0 (type: string)"}
                                        |  outputColumnNames:["_col2","_col5"]
                                        |  Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-                                       |<-Map 23 [SIMPLE_EDGE]
-                                       |  Reduce Output Operator [RS_58]
+                                       |<-Map 12 [SIMPLE_EDGE]
+                                       |  Reduce Output Operator [RS_23]
                                        |     key expressions:_col0 (type: string)
                                        |     Map-reduce partition columns:_col0 (type: string)
                                        |     sort order:+
                                        |     Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                                        |     value expressions:_col1 (type: string)
-                                       |     Select Operator [SEL_49]
+                                       |     Select Operator [SEL_14]
                                        |        outputColumnNames:["_col0","_col1"]
                                        |        Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                                       |        Filter Operator [FIL_155]
+                                       |        Filter Operator [FIL_150]
                                        |           predicate:key is not null (type: boolean)
                                        |           Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                                       |           TableScan [TS_48]
+                                       |           TableScan [TS_13]
                                        |              alias:y
                                        |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                       |<-Reducer 18 [SIMPLE_EDGE]
-                                          Reduce Output Operator [RS_56]
+                                       |<-Reducer 4 [SIMPLE_EDGE]
+                                          Reduce Output Operator [RS_21]
                                              key expressions:_col2 (type: string)
                                              Map-reduce partition columns:_col2 (type: string)
                                              sort order:+
-                                             Statistics:Num rows: 209 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
-                                             Merge Join Operator [MERGEJOIN_164]
+                                             Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
+                                             Merge Join Operator [MERGEJOIN_162]
                                              |  condition map:[{"":"Inner Join 0 to 1"}]
-                                             |  keys:{"1":"_col1 (type: string)","0":"_col1 (type: string)"}
+                                             |  keys:{"0":"_col1 (type: string)","1":"_col1 (type: string)"}
                                              |  outputColumnNames:["_col2"]
-                                             |  Statistics:Num rows: 209 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
-                                             |<-Map 22 [SIMPLE_EDGE]
-                                             |  Reduce Output Operator [RS_53]
+                                             |  Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
+                                             |<-Map 11 [SIMPLE_EDGE]
+                                             |  Reduce Output Operator [RS_18]
                                              |     key expressions:_col1 (type: string)
                                              |     Map-reduce partition columns:_col1 (type: string)
                                              |     sort order:+
                                              |     Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
                                              |     value expressions:_col0 (type: string)
-                                             |     Select Operator [SEL_47]
+                                             |     Select Operator [SEL_12]
                                              |        outputColumnNames:["_col0","_col1"]
                                              |        Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
-                                             |        Filter Operator [FIL_154]
+                                             |        Filter Operator [FIL_149]
                                              |           predicate:(value is not null and key is not null) (type: boolean)
                                              |           Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
-                                             |           TableScan [TS_46]
+                                             |           TableScan [TS_11]
                                              |              alias:x
                                              |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                                             |<-Reducer 17 [SIMPLE_EDGE]
-                                                Reduce Output Operator [RS_51]
+                                             |<-Reducer 3 [SIMPLE_EDGE]
+                                                Reduce Output Operator [RS_16]
                                                    key expressions:_col1 (type: string)
                                                    Map-reduce partition columns:_col1 (type: string)
                                                    sort order:+
-                                                   Statistics:Num rows: 190 Data size: 2008 Basic stats: COMPLETE Column stats: NONE
-                                                   Select Operator [SEL_45]
+                                                   Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
+                                                   Select Operator [SEL_10]
                                                       outputColumnNames:["_col1"]
-                                                      Statistics:Num rows: 190 Data size: 2008 Basic stats: COMPLETE Column stats: NONE
-                                                      Group By Operator [GBY_44]
+                                                      Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
+                                                      Group By Operator [GBY_9]
                                                       |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
                                                       |  outputColumnNames:["_col0","_col1"]
-                                                      |  Statistics:Num rows: 190 Data size: 2008 Basic stats: COMPLETE Column stats: NONE
-                                                      |<-Union 16 [SIMPLE_EDGE]
-                                                         |<-Map 21 [CONTAINS]
-                                                         |  Reduce Output Operator [RS_43]
+                                                      |  Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
+                                                      |<-Union 2 [SIMPLE_EDGE]
+                                                         |<-Map 1 [CONTAINS]
+                                                         |  Reduce Output Operator [RS_8]
                                                          |     key expressions:_col0 (type: string), _col1 (type: string)
                                                          |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
                                                          |     sort order:++
-                                                         |     Statistics:Num rows: 381 Data size: 4028 Basic stats: COMPLETE Column stats: NONE
-                                                         |     Group By Operator [GBY_42]
+                                                         |     Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
+                                                         |     Group By Operator [GBY_7]
                                                          |        keys:_col0 (type: string), _col1 (type: string)
                                                          |        outputColumnNames:["_col0","_col1"]
-                                                         |        Statistics:Num rows: 381 Data size: 4028 Basic stats: COMPLETE Column stats: NONE
-                                                         |        Select Operator [SEL_38]
+                                                         |        Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
+                                                         |        Select Operator [SEL_1]
                                                          |           outputColumnNames:["_col0","_col1"]
-                                                         |           Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                                                         |           Filter Operator [FIL_153]
+                                                         |           Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+                                                         |           Filter Operator [FIL_147]
                                                          |              predicate:value is not null (type: boolean)
-                                                         |              Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                                                         |              TableScan [TS_37]
-                                                         |                 alias:y
-                                                         |                 Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                                                         |<-Reducer 15 [CONTAINS]
-                                                            Reduce Output Operator [RS_43]
+                                                         |              Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+                                                         |              TableScan [TS_0]
+                                                         |                 alias:x
+                                                         |                 Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                                                         |<-Map 10 [CONTAINS]
+                                                            Reduce Output Operator [RS_8]
                                                                key expressions:_col0 (type: string), _col1 (type: string)
                                                                Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
                                                                sort order:++
-                                                               Statistics:Num rows: 381 Data size: 4028 Basic stats: COMPLETE Column stats: NONE
-                                                               Group By Operator [GBY_42]
+                                                               Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
+                                                               Group By Operator [GBY_7]
                                                                   keys:_col0 (type: string), _col1 (type: string)
                                                                   outputColumnNames:["_col0","_col1"]
-                                                                  Statistics:Num rows: 381 Data size: 4028 Basic stats: COMPLETE Column stats: NONE
-                                                                  Group By Operator [GBY_35]
-                                                                  |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
-                                                                  |  outputColumnNames:["_col0","_col1"]
-                                                                  |  Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-                                                                  |<-Union 14 [SIMPLE_EDGE]
-                                                                     |<-Map 13 [CONTAINS]
-                                                                     |  Reduce Output Operator [RS_34]
-                                                                     |     key expressions:_col0 (type: string), _col1 (type: string)
-                                                                     |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
-                                                                     |     sort order:++
-                                                                     |     Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                                                                     |     Group By Operator [GBY_33]
-                                                                     |        keys:_col0 (type: string), _col1 (type: string)
-                                                                     |        outputColumnNames:["_col0","_col1"]
-                                                                     |        Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                                                                     |        Select Operator [SEL_27]
-                                                                     |           outputColumnNames:["_col0","_col1"]
-                                                                     |           Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                                                                     |           Filter Operator [FIL_151]
-                                                                     |              predicate:value is not null (type: boolean)
-                                                                     |              Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
-                                                                     |              TableScan [TS_26]
-                                                                     |                 alias:x
-                                                                     |                 Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                                                                     |<-Map 20 [CONTAINS]
-                                                                        Reduce Output Operator [RS_34]
-                                                                           key expressions:_col0 (type: string), _col1 (type: string)
-                                                                           Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
-                                                                           sort order:++
-                                                                           Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                                                                           Group By Operator [GBY_33]
-                                                                              keys:_col0 (type: string), _col1 (type: string)
-                                                                              outputColumnNames:["_col0","_col1"]
-                                                                              Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-                                                                              Select Operator [SEL_29]
-                                                                                 outputColumnNames:["_col0","_col1"]
-                                                                                 Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                                                                                 Filter Operator [FIL_152]
-                                                                                    predicate:value is not null (type: boolean)
-                                                                                    Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-                                                                                    TableScan [TS_28]
-                                                                                       alias:y
-                                                                                       Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                                                                  Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
+                                                                  Select Operator [SEL_3]
+                                                                     outputColumnNames:["_col0","_col1"]
+                                                                     Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                                                                     Filter Operator [FIL_148]
+                                                                        predicate:value is not null (type: boolean)
+                                                                        Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                                                                        TableScan [TS_2]
+                                                                           alias:y
+                                                                           Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 
 PREHOOK: query: EXPLAIN
 SELECT x.key, z.value, y.value
@@ -1396,7 +1396,7 @@ Stage-0
          File Output Operator [FS_18]
             compressed:false
             Statistics:Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Select Operator [SEL_17]
                outputColumnNames:["_col0","_col1","_col2"]
                Statistics:Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
@@ -1525,7 +1525,7 @@ Stage-0
          File Output Operator [FS_69]
             compressed:false
             Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Limit [LIM_68]
                Number of rows:100
                Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
@@ -1605,7 +1605,7 @@ Stage-0
                                        Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
                                        Map Join Operator [MAPJOIN_110]
                                        |  condition map:[{"":"Inner Join 0 to 1"}]
-                                       |  keys:{"Map 10":"_col2 (type: string), _col4 (type: string)","Map 3":"_col4 (type: string), _col6 (type: string)"}
+                                       |  keys:{"Map 3":"_col4 (type: string), _col6 (type: string)","Map 10":"_col2 (type: string), _col4 (type: string)"}
                                        |  outputColumnNames:["_col2","_col3","_col14","_col15","_col17"]
                                        |  Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
                                        |<-Map 10 [BROADCAST_EDGE]
@@ -1617,7 +1617,7 @@ Stage-0
                                        |     value expressions:_col3 (type: string), _col5 (type: string)
                                        |     Map Join Operator [MAPJOIN_109]
                                        |     |  condition map:[{"":"Inner Join 0 to 1"}]
-                                       |     |  keys:{"Map 10":"_col0 (type: string)","Map 9":"_col0 (type: string)"}
+                                       |     |  keys:{"Map 9":"_col0 (type: string)","Map 10":"_col0 (type: string)"}
                                        |     |  outputColumnNames:["_col2","_col3","_col4","_col5"]
                                        |     |  Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
                                        |     |<-Map 9 [BROADCAST_EDGE]
@@ -1755,7 +1755,7 @@ Stage-0
          File Output Operator [FS_59]
             compressed:false
             Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_57]
             |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
             |  outputColumnNames:["_col0","_col1"]
@@ -2022,7 +2022,7 @@ Stage-0
          File Output Operator [FS_119]
             compressed:false
             Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Group By Operator [GBY_117]
             |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
             |  outputColumnNames:["_col0","_col1"]
@@ -2043,7 +2043,7 @@ Stage-0
                |           Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
                |           Map Join Operator [MAPJOIN_167]
                |           |  condition map:[{"":"Inner Join 0 to 1"}]
-               |           |  keys:{"Map 31":"_col0 (type: string)","Reducer 26":"_col2 (type: string)"}
+               |           |  keys:{"Reducer 26":"_col2 (type: string)","Map 31":"_col0 (type: string)"}
                |           |  outputColumnNames:["_col2","_col5"]
                |           |  Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
                |           |<-Reducer 26 [BROADCAST_EDGE]
@@ -2054,7 +2054,7 @@ Stage-0
                |           |     Statistics:Num rows: 242 Data size: 2565 Basic stats: COMPLETE Column stats: NONE
                |           |     Map Join Operator [MAPJOIN_166]
                |           |     |  condition map:[{"":"Inner Join 0 to 1"}]
-               |           |     |  keys:{"Map 30":"_col1 (type: string)","Reducer 26":"_col1 (type: string)"}
+               |           |     |  keys:{"Reducer 26":"_col1 (type: string)","Map 30":"_col1 (type: string)"}
                |           |     |  outputColumnNames:["_col2"]
                |           |     |  Statistics:Num rows: 242 Data size: 2565 Basic stats: COMPLETE Column stats: NONE
                |           |     |<-Map 30 [BROADCAST_EDGE]
@@ -2081,7 +2081,7 @@ Stage-0
                |           |           |  outputColumnNames:["_col0","_col1"]
                |           |           |  Statistics:Num rows: 220 Data size: 2332 Basic stats: COMPLETE Column stats: NONE
                |           |           |<-Union 25 [SIMPLE_EDGE]
-               |           |              |<-Reducer 24 [CONTAINS]
+               |           |              |<-Map 29 [CONTAINS]
                |           |              |  Reduce Output Operator [RS_94]
                |           |              |     key expressions:_col0 (type: string), _col1 (type: string)
                |           |              |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
@@ -2091,84 +2091,16 @@ Stage-0
                |           |              |        keys:_col0 (type: string), _col1 (type: string)
                |           |              |        outputColumnNames:["_col0","_col1"]
                |           |              |        Statistics:Num rows: 440 Data size: 4664 Basic stats: COMPLETE Column stats: NONE
-               |           |              |        Group By Operator [GBY_86]
-               |           |              |        |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
-               |           |              |        |  outputColumnNames:["_col0","_col1"]
-               |           |              |        |  Statistics:Num rows: 190 Data size: 2008 Basic stats: COMPLETE Column stats: NONE
-               |           |              |        |<-Union 23 [SIMPLE_EDGE]
-               |           |              |           |<-Reducer 22 [CONTAINS]
-               |           |              |           |  Reduce Output Operator [RS_85]
-               |           |              |           |     key expressions:_col0 (type: string), _col1 (type: string)
-               |           |              |           |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
-               |           |              |           |     sort order:++
-               |           |              |           |     Statistics:Num rows: 381 Data size: 4028 Basic stats: COMPLETE Column stats: NONE
-               |           |              |           |     Group By Operator [GBY_84]
-               |           |              |           |        keys:_col0 (type: string), _col1 (type: string)
-               |           |              |           |        outputColumnNames:["_col0","_col1"]
-               |           |              |           |        Statistics:Num rows: 381 Data size: 4028 Basic stats: COMPLETE Column stats: NONE
-               |           |              |           |        Group By Operator [GBY_77]
-               |           |              |           |        |  keys:KEY._col0 (type: string), KEY._col1 (type: string)
-               |           |              |           |        |  outputColumnNames:["_col0","_col1"]
-               |           |              |           |        |  Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-               |           |              |           |        |<-Union 21 [SIMPLE_EDGE]
-               |           |              |           |           |<-Map 20 [CONTAINS]
-               |           |              |           |           |  Reduce Output Operator [RS_76]
-               |           |              |           |           |     key expressions:_col0 (type: string), _col1 (type: string)
-               |           |              |           |           |     Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
-               |           |              |           |           |     sort order:++
-               |           |              |           |           |     Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-               |           |              |           |           |     Group By Operator [GBY_75]
-               |           |              |           |           |        keys:_col0 (type: string), _col1 (type: string)
-               |           |              |           |           |        outputColumnNames:["_col0","_col1"]
-               |           |              |           |           |        Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
-               |           |              |           |           |        Select Operator [SEL_69]
-               |           | 

<TRUNCATED>