You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2020/02/27 16:25:48 UTC

[hive] branch master updated: HIVE-22893: Enhance data size estimation for fields computed by UDFs (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)

This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new ffba5d6  HIVE-22893: Enhance data size estimation for fields computed by UDFs (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)
ffba5d6 is described below

commit ffba5d6cd97d26c12361e429b09a958b41da421a
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Thu Feb 27 16:20:05 2020 +0000

    HIVE-22893: Enhance data size estimation for fields computed by UDFs (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)
    
    Close apache/hive#915
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |   3 +
 .../clientpositive/udaf_example_group_concat.q.out |   8 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java    |  92 +++++++++++--
 .../stats/estimator/PessimisticStatCombiner.java   |  49 +++++++
 .../hive/ql/stats/estimator/StatEstimator.java     |  43 ++++++
 .../ql/stats/estimator/StatEstimatorProvider.java  |  29 ++++
 .../org/apache/hadoop/hive/ql/udf/UDFSubstr.java   |  52 ++++++-
 .../hadoop/hive/ql/udf/generic/GenericUDF.java     |   9 ++
 .../hive/ql/udf/generic/GenericUDFBridge.java      |  14 +-
 .../hadoop/hive/ql/udf/generic/GenericUDFCase.java |  30 ++++-
 .../hive/ql/udf/generic/GenericUDFCoalesce.java    |  25 +++-
 .../hadoop/hive/ql/udf/generic/GenericUDFIf.java   |  27 +++-
 ql/src/test/queries/clientpositive/udf_coalesce.q  |   7 +-
 .../cbo_rp_gby2_map_multi_distinct.q.out           |  28 ++--
 .../cbo_rp_groupby3_noskew_multi_distinct.q.out    |   4 +-
 .../clientpositive/constprog_when_case.q.out       |   4 +-
 .../clientpositive/count_dist_rewrite.q.out        |   6 +-
 ql/src/test/results/clientpositive/groupby11.q.out |  14 +-
 .../test/results/clientpositive/groupby2_map.q.out |  12 +-
 .../groupby2_map_multi_distinct.q.out              |  24 ++--
 .../results/clientpositive/groupby2_map_skew.q.out |  16 +--
 .../results/clientpositive/groupby2_noskew.q.out   |  10 +-
 .../groupby2_noskew_multi_distinct.q.out           |  10 +-
 .../test/results/clientpositive/groupby3_map.q.out |   4 +-
 .../groupby3_map_multi_distinct.q.out              |   4 +-
 .../results/clientpositive/groupby3_map_skew.q.out |   4 +-
 ql/src/test/results/clientpositive/groupby4.q.out  |  12 +-
 .../results/clientpositive/groupby4_noskew.q.out   |   8 +-
 ql/src/test/results/clientpositive/groupby6.q.out  |  12 +-
 .../test/results/clientpositive/groupby6_map.q.out |  10 +-
 .../results/clientpositive/groupby6_map_skew.q.out |  14 +-
 .../results/clientpositive/groupby6_noskew.q.out   |   8 +-
 .../results/clientpositive/groupby8_map_skew.q.out |   8 +-
 ql/src/test/results/clientpositive/groupby9.q.out  |  32 ++---
 .../results/clientpositive/groupby_map_ppr.q.out   |  12 +-
 .../groupby_map_ppr_multi_distinct.q.out           |  12 +-
 .../groupby_multi_single_reducer.q.out             |  66 ++++-----
 .../groupby_multi_single_reducer2.q.out            |  20 +--
 .../clientpositive/groupby_nocolumnalign.q.out     |   4 +-
 .../results/clientpositive/groupby_position.q.out  |  32 ++---
 .../test/results/clientpositive/groupby_ppr.q.out  |  10 +-
 .../groupby_ppr_multi_distinct.q.out               |  10 +-
 .../infer_bucket_sort_dyn_part.q.out               | 149 +++++++++------------
 .../results/clientpositive/list_bucket_dml_6.q.out |  32 ++---
 .../results/clientpositive/list_bucket_dml_7.q.out |  32 ++---
 .../results/clientpositive/list_bucket_dml_8.q.out |  16 +--
 .../clientpositive/llap/count_dist_rewrite.q.out   |   6 +-
 .../clientpositive/llap/explainuser_2.q.out        |  24 ++--
 .../results/clientpositive/llap/groupby2.q.out     |  14 +-
 .../llap/tez_union_multiinsert.q.out               |  94 ++++++-------
 .../results/clientpositive/llap/udf_coalesce.q.out | 115 +++++++++++++---
 .../clientpositive/llap/unionDistinct_1.q.out      |   6 +-
 .../clientpositive/llap/vector_case_when_1.q.out   |  12 +-
 .../clientpositive/llap/vector_case_when_2.q.out   |  24 ++--
 .../clientpositive/llap/vector_groupby4.q.out      |  12 +-
 .../clientpositive/llap/vector_groupby6.q.out      |  12 +-
 .../clientpositive/llap/vector_if_expr.q.out       |   8 +-
 .../results/clientpositive/llap/vector_nvl.q.out   |   6 +-
 .../results/clientpositive/llap/vector_udf1.q.out  |   6 +-
 .../llap/vectorized_string_funcs.q.out             |   4 +-
 .../llap/vectorized_timestamp_funcs.q.out          |   8 +-
 .../clientpositive/merge_dynamic_partition4.q.out  |  78 ++++-------
 .../clientpositive/merge_dynamic_partition5.q.out  |  78 ++++-------
 .../clientpositive/nullgroup4_multi_distinct.q.out |   4 +-
 .../offset_limit_global_optimizer.q.out            |  80 +++++------
 .../perf/tez/constraints/query19.q.out             |  14 +-
 .../perf/tez/constraints/query79.q.out             |  12 +-
 .../perf/tez/constraints/query8.q.out              |  20 +--
 .../perf/tez/constraints/query85.q.out             |   6 +-
 .../perf/tez/constraints/query99.q.out             |  18 +--
 .../results/clientpositive/perf/tez/query19.q.out  |  14 +-
 .../results/clientpositive/perf/tez/query23.q.out  |  14 +-
 .../results/clientpositive/perf/tez/query79.q.out  |  12 +-
 .../results/clientpositive/perf/tez/query8.q.out   |  20 +--
 .../results/clientpositive/perf/tez/query85.q.out  |   6 +-
 .../results/clientpositive/perf/tez/query99.q.out  |  18 +--
 .../results/clientpositive/spark/union17.q.out     |  16 +--
 ql/src/test/results/clientpositive/union17.q.out   |  14 +-
 .../clientpositive/vector_case_when_1.q.out        |  12 +-
 .../clientpositive/vector_case_when_2.q.out        |  24 ++--
 .../results/clientpositive/vector_groupby4.q.out   |  12 +-
 .../results/clientpositive/vector_groupby6.q.out   |  12 +-
 .../results/clientpositive/vector_if_expr.q.out    |   8 +-
 .../test/results/clientpositive/vector_nvl.q.out   |   6 +-
 .../clientpositive/vectorization_multi_value.q.out |  32 ++---
 .../clientpositive/vectorized_string_funcs.q.out   |   4 +-
 .../vectorized_timestamp_funcs.q.out               |   8 +-
 .../hadoop/hive/metastore/MetaStoreSchemaInfo.java |   9 +-
 88 files changed, 1142 insertions(+), 787 deletions(-)

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 bfc2695..1a4d71b 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2519,6 +2519,9 @@ public class HiveConf extends Configuration {
                     "higher compute cost. (NDV means the number of distinct values.). It only affects the FM-Sketch \n" +
                     "(not the HLL algorithm which is the default), where it computes the number of necessary\n" +
                     " bitvectors to achieve the accuracy."),
+    HIVE_STATS_ESTIMATORS_ENABLE("hive.stats.estimators.enable", true,
+        "Estimators are able to provide more accurate column statistic infos for UDF results."),
+
     /**
      * @deprecated Use MetastoreConf.STATS_NDV_TUNER
      */
diff --git a/contrib/src/test/results/clientpositive/udaf_example_group_concat.q.out b/contrib/src/test/results/clientpositive/udaf_example_group_concat.q.out
index 15dd4c0..509b016 100644
--- a/contrib/src/test/results/clientpositive/udaf_example_group_concat.q.out
+++ b/contrib/src/test/results/clientpositive/udaf_example_group_concat.q.out
@@ -39,13 +39,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 250 Data size: 526000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 501250 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   null sort order: z
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 526000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 501250 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: array<string>)
       Reduce Operator Tree:
         Group By Operator
@@ -53,10 +53,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 250 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 67250 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 250 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 67250 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
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 cb2d0a7..bd4a4f6 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
@@ -27,6 +27,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -73,6 +74,9 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.Statistics;
 import org.apache.hadoop.hive.ql.plan.Statistics.State;
 import org.apache.hadoop.hive.ql.stats.BasicStats.Factory;
+import org.apache.hadoop.hive.ql.stats.estimator.StatEstimator;
+import org.apache.hadoop.hive.ql.stats.estimator.StatEstimatorProvider;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.apache.hadoop.hive.ql.udf.generic.NDV;
@@ -81,6 +85,7 @@ import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.StandardConstantListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StandardConstantMapObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StandardConstantStructObjectInspector;
@@ -1528,18 +1533,7 @@ public class StatsUtils {
         return null;
       }
     } else if (end instanceof ExprNodeConstantDesc) {
-
-      // constant projection
-      ExprNodeConstantDesc encd = (ExprNodeConstantDesc) end;
-
-      colName = encd.getName();
-      colType = encd.getTypeString();
-      if (encd.getValue() == null) {
-        // null projection
-        numNulls = numRows;
-      } else {
-        countDistincts = 1;
-      }
+      return buildColStatForConstant(conf, numRows, (ExprNodeConstantDesc) end);
     } else if (end instanceof ExprNodeGenericFuncDesc) {
       ExprNodeGenericFuncDesc engfd = (ExprNodeGenericFuncDesc) end;
       colName = engfd.getName();
@@ -1560,6 +1554,30 @@ public class StatsUtils {
         }
       }
 
+      if (conf.getBoolVar(ConfVars.HIVE_STATS_ESTIMATORS_ENABLE)) {
+        Optional<StatEstimatorProvider> sep = engfd.getGenericUDF().adapt(StatEstimatorProvider.class);
+        if (sep.isPresent()) {
+          StatEstimator se = sep.get().getStatEstimator();
+          List<ColStatistics> csList = new ArrayList<ColStatistics>();
+          for (ExprNodeDesc child : engfd.getChildren()) {
+            ColStatistics cs = getColStatisticsFromExpression(conf, parentStats, child);
+            if (cs == null) {
+              break;
+            }
+            csList.add(cs);
+          }
+          if (csList.size() == engfd.getChildren().size()) {
+            Optional<ColStatistics> res = se.estimate(csList);
+            if (res.isPresent()) {
+              ColStatistics newStats = res.get();
+              colType = colType.toLowerCase();
+              newStats.setColumnType(colType);
+              newStats.setColumnName(colName);
+              return newStats;
+            }
+          }
+        }
+      }
       // fallback to default
       countDistincts = getNDVFor(engfd, numRows, parentStats);
     } else if (end instanceof ExprNodeColumnListDesc) {
@@ -1590,6 +1608,56 @@ public class StatsUtils {
     return colStats;
   }
 
+  private static ColStatistics buildColStatForConstant(HiveConf conf, long numRows, ExprNodeConstantDesc encd) {
+
+    long numNulls = 0;
+    long countDistincts = 0;
+    if (encd.getValue() == null) {
+      // null projection
+      numNulls = numRows;
+    } else {
+      countDistincts = 1;
+    }
+    String colType = encd.getTypeString();
+    colType = colType.toLowerCase();
+    ObjectInspector oi = encd.getWritableObjectInspector();
+    double avgColSize = getAvgColLenOf(conf, oi, colType);
+    ColStatistics colStats = new ColStatistics(encd.getName(), colType);
+    colStats.setAvgColLen(avgColSize);
+    colStats.setCountDistint(countDistincts);
+    colStats.setNumNulls(numNulls);
+
+    Optional<Number> value = getConstValue(encd);
+    if (value.isPresent()) {
+      colStats.setRange(value.get(), value.get());
+    }
+    return colStats;
+  }
+
+  private static Optional<Number> getConstValue(ExprNodeConstantDesc encd) {
+    if (encd.getValue() != null) {
+      String constant = encd.getValue().toString();
+      PrimitiveCategory category = GenericUDAFSum.getReturnType(encd.getTypeInfo());
+      try {
+        switch (category) {
+        case INT:
+        case BYTE:
+        case SHORT:
+        case LONG:
+          return Optional.of(Long.parseLong(constant));
+        case FLOAT:
+        case DOUBLE:
+        case DECIMAL:
+          return Optional.of(Double.parseDouble(constant));
+        default:
+        }
+      } catch (Exception e) {
+        LOG.debug("Interpreting constant (" + constant + ")  resulted in exception", e);
+      }
+    }
+    return Optional.empty();
+  }
+
   private static boolean isWideningCast(ExprNodeGenericFuncDesc engfd) {
     GenericUDF udf = engfd.getGenericUDF();
     if (!FunctionRegistry.isOpCast(udf)) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/estimator/PessimisticStatCombiner.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/estimator/PessimisticStatCombiner.java
new file mode 100644
index 0000000..131b422
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/estimator/PessimisticStatCombiner.java
@@ -0,0 +1,49 @@
+package org.apache.hadoop.hive.ql.stats.estimator;
+
+import java.util.Optional;
+
+import org.apache.hadoop.hive.ql.plan.ColStatistics;
+
+/**
+ * Combines {@link ColStatistics} objects to provide the most pessimistic estimate.
+ */
+public class PessimisticStatCombiner {
+
+  private boolean inited;
+  private ColStatistics result;
+
+  public void add(ColStatistics stat) {
+    if (!inited) {
+      inited = true;
+      result = stat.clone();
+      result.setRange(null);
+      result.setIsEstimated(true);
+      return;
+    } else {
+      if (stat.getAvgColLen() > result.getAvgColLen()) {
+        result.setAvgColLen(stat.getAvgColLen());
+      }
+      if (stat.getCountDistint() > result.getCountDistint()) {
+        result.setCountDistint(stat.getCountDistint());
+      }
+      if (stat.getNumNulls() > result.getNumNulls()) {
+        result.setNumNulls(stat.getNumNulls());
+      }
+      if (stat.getNumTrues() > result.getNumTrues()) {
+        result.setNumTrues(stat.getNumTrues());
+      }
+      if (stat.getNumFalses() > result.getNumFalses()) {
+        result.setNumFalses(stat.getNumFalses());
+      }
+      if (stat.isFilteredColumn()) {
+        result.setFilterColumn();
+      }
+
+    }
+
+  }
+  public Optional<ColStatistics> getResult() {
+    return Optional.of(result);
+
+  }
+}
\ No newline at end of file
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/estimator/StatEstimator.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/estimator/StatEstimator.java
new file mode 100644
index 0000000..d1fc3f2
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/estimator/StatEstimator.java
@@ -0,0 +1,43 @@
+/*
+ * 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.stats.estimator;
+
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.hadoop.hive.ql.plan.ColStatistics;
+
+/**
+ * Enables statistics related computation on UDFs
+ */
+public interface StatEstimator {
+
+  /**
+   * Computes the output statistics of the actual UDF.
+   *
+   * The estimator should return with a prefereably overestimated {@link ColStatistics} object if possible.
+   * The actual estimation logic may decide to not give an estimation; it should return with {@link Optional#empty()}.
+   *
+   * Note: at the time of the call there will be {@link ColStatistics} for all the arguments; if that is not available - the estimation is skipped.
+   *
+   * @param argStats the statistics for every argument of the UDF
+   * @return {@link ColStatistics} estimate for the actual UDF.
+   */
+  public Optional<ColStatistics> estimate(List<ColStatistics> argStats);
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/estimator/StatEstimatorProvider.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/estimator/StatEstimatorProvider.java
new file mode 100644
index 0000000..96865d1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/estimator/StatEstimatorProvider.java
@@ -0,0 +1,29 @@
+/*
+ * 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.stats.estimator;
+
+/**
+ * Marker interface for UDFs to communicate that the usage of StatEstimators is supported by the UDF.
+ */
+public interface StatEstimatorProvider {
+
+  /**
+   * Returns the {@link StatEstimator} for the given UDF instance.
+   */
+  public StatEstimator getStatEstimator();
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFSubstr.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFSubstr.java
index 5b1964c..cb28cf6 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFSubstr.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFSubstr.java
@@ -19,12 +19,18 @@
 package org.apache.hadoop.hive.ql.udf;
 
 import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
 
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDF;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringSubstrColStart;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringSubstrColStartLen;
+import org.apache.hadoop.hive.ql.plan.ColStatistics;
+import org.apache.hadoop.hive.ql.plan.ColStatistics.Range;
+import org.apache.hadoop.hive.ql.stats.estimator.StatEstimator;
+import org.apache.hadoop.hive.ql.stats.estimator.StatEstimatorProvider;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
@@ -48,7 +54,7 @@ import org.apache.hadoop.io.Text;
     + "  > SELECT _FUNC_('Facebook', 5, 1) FROM src LIMIT 1;\n"
     + "  'b'")
 @VectorizedExpressions({StringSubstrColStart.class, StringSubstrColStartLen.class})
-public class UDFSubstr extends UDF {
+public class UDFSubstr extends UDF implements StatEstimatorProvider {
 
   private final int[] index;
   private final Text r;
@@ -131,4 +137,48 @@ public class UDFSubstr extends UDF {
   public BytesWritable evaluate(BytesWritable bw, IntWritable pos){
     return evaluate(bw, pos, maxValue);
   }
+
+  @Override
+  public StatEstimator getStatEstimator() {
+    return new SubStrStatEstimator();
+  }
+
+  private static class SubStrStatEstimator implements StatEstimator {
+
+    @Override
+    public Optional<ColStatistics> estimate(List<ColStatistics> csList) {
+      ColStatistics cs = csList.get(0).clone();
+      // this might bad in a skewed case; consider:
+      // 1 row with 1000 long string
+      // 99 rows with 0 length
+      // orig avg is 10
+      // new avg is 5 (if substr(5)) ; but in reality it will stay ~10
+      Optional<Double> start = getRangeWidth(csList.get(1).getRange());
+      Range startRange = csList.get(1).getRange();
+      if (startRange != null && startRange.minValue != null) {
+        double newAvgColLen = cs.getAvgColLen() - startRange.minValue.doubleValue();
+        if (newAvgColLen > 0) {
+          cs.setAvgColLen(newAvgColLen);
+        }
+      }
+      if (csList.size() > 2) {
+        Range lengthRange = csList.get(2).getRange();
+        if (lengthRange != null && lengthRange.maxValue != null) {
+          Double w = lengthRange.maxValue.doubleValue();
+          if (cs.getAvgColLen() > w) {
+            cs.setAvgColLen(w);
+          }
+        }
+      }
+      return Optional.of(cs);
+    }
+
+    private Optional<Double> getRangeWidth(Range range) {
+      if (range.minValue != null && range.maxValue != null) {
+        return Optional.of(range.maxValue.doubleValue() - range.minValue.doubleValue());
+      }
+      return Optional.empty();
+    }
+
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
index 6597f4b..c1bf325 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.udf.generic;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Optional;
 
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
@@ -638,4 +639,12 @@ public abstract class GenericUDF implements Closeable {
       return i + ORDINAL_SUFFIXES[i % 10];
     }
   }
+
+  @SuppressWarnings("unchecked")
+  public <T> Optional<T> adapt(Class<T> clazz) {
+    if (clazz.isInstance(this)) {
+      return Optional.of((T) this);
+    }
+    return Optional.empty();
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
index 7a644fc..0a275ae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.udf.generic;
 import java.io.Serializable;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
+import java.util.Optional;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
@@ -97,7 +98,7 @@ public class GenericUDFBridge extends GenericUDF implements Serializable {
     this.isOperator = isOperator;
     this.udfClassName = udfClassName;
   }
- 
+
   // For Java serialization only
   public GenericUDFBridge() {
   }
@@ -151,7 +152,7 @@ public class GenericUDFBridge extends GenericUDF implements Serializable {
   public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
 
     try {
-      udf = (UDF)getUdfClassInternal().newInstance();
+      udf = getUdfClassInternal().newInstance();
     } catch (Exception e) {
       throw new UDFArgumentException(
           "Unable to instantiate UDF implementation class " + udfClassName + ": " + e);
@@ -249,4 +250,13 @@ public class GenericUDFBridge extends GenericUDF implements Serializable {
   public interface UdfWhitelistChecker {
     boolean isUdfAllowed(Class<?> clazz);
   }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T> Optional<T> adapt(Class<T> clazz) {
+    if (clazz.isInstance(udf)) {
+      return Optional.of((T) udf);
+    }
+    return super.adapt(clazz);
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCase.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCase.java
index 06e9d00..2857835 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCase.java
@@ -18,16 +18,23 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
+import java.util.List;
+import java.util.Optional;
+
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ColStatistics;
+import org.apache.hadoop.hive.ql.stats.estimator.StatEstimator;
+import org.apache.hadoop.hive.ql.stats.estimator.StatEstimatorProvider;
+import org.apache.hadoop.hive.ql.stats.estimator.PessimisticStatCombiner;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 
 /**
  * GenericUDF Class for SQL construct "CASE a WHEN b THEN c [ELSE f] END".
- * 
+ *
  * NOTES: 1. a and b should be compatible, or an exception will be
  * thrown. 2. c and f should be compatible types, or an exception will be
  * thrown.
@@ -49,7 +56,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
     + " END\n"
     + " FROM emp_details")
 
-public class GenericUDFCase extends GenericUDF {
+public class GenericUDFCase extends GenericUDF implements StatEstimatorProvider {
   private transient ObjectInspector[] argumentOIs;
   private transient GenericUDFUtils.ReturnObjectInspectorResolver returnOIResolver;
   private transient GenericUDFUtils.ReturnObjectInspectorResolver caseOIResolver;
@@ -138,4 +145,23 @@ public class GenericUDFCase extends GenericUDF {
     return sb.toString();
   }
 
+  @Override
+  public StatEstimator getStatEstimator() {
+    return new CaseStatEstimator();
+  }
+
+  static class CaseStatEstimator implements StatEstimator {
+
+    @Override
+    public Optional<ColStatistics> estimate(List<ColStatistics> argStats) {
+      PessimisticStatCombiner combiner = new PessimisticStatCombiner();
+      for (int i = 1; i < argStats.size(); i += 2) {
+        combiner.add(argStats.get(i));
+      }
+      combiner.add(argStats.get(argStats.size() - 1));
+      return combiner.getResult();
+    }
+  }
+
+
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCoalesce.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCoalesce.java
index 8ebe9e0..a598905 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCoalesce.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCoalesce.java
@@ -18,9 +18,16 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
+import java.util.List;
+import java.util.Optional;
+
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ColStatistics;
+import org.apache.hadoop.hive.ql.stats.estimator.StatEstimator;
+import org.apache.hadoop.hive.ql.stats.estimator.StatEstimatorProvider;
+import org.apache.hadoop.hive.ql.stats.estimator.PessimisticStatCombiner;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
 /**
@@ -33,7 +40,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
     value = "_FUNC_(a1, a2, ...) - Returns the first non-null argument",
     extended = "Example:\n"
     + "  > SELECT _FUNC_(NULL, 1, NULL) FROM src LIMIT 1;\n" + "  1")
-public class GenericUDFCoalesce extends GenericUDF {
+public class GenericUDFCoalesce extends GenericUDF implements StatEstimatorProvider {
   private transient ObjectInspector[] argumentOIs;
   private transient GenericUDFUtils.ReturnObjectInspectorResolver returnOIResolver;
 
@@ -72,4 +79,20 @@ public class GenericUDFCoalesce extends GenericUDF {
     return getStandardDisplayString("COALESCE", children, ",");
   }
 
+  @Override
+  public StatEstimator getStatEstimator() {
+    return new CoalesceStatEstimator();
+  }
+
+  static class CoalesceStatEstimator implements StatEstimator {
+
+    @Override
+    public Optional<ColStatistics> estimate(List<ColStatistics> argStats) {
+      PessimisticStatCombiner combiner = new PessimisticStatCombiner();
+      for (int i = 0; i < argStats.size(); i++) {
+        combiner.add(argStats.get(i));
+      }
+      return combiner.getResult();
+    }
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
index 23708dc..eaa3523 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
+import java.util.List;
+import java.util.Optional;
+
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
@@ -25,6 +28,10 @@ import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ColStatistics;
+import org.apache.hadoop.hive.ql.stats.estimator.StatEstimator;
+import org.apache.hadoop.hive.ql.stats.estimator.StatEstimatorProvider;
+import org.apache.hadoop.hive.ql.stats.estimator.PessimisticStatCombiner;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
@@ -115,7 +122,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprVarCharScalarStri
   IfExprTimestampScalarColumn.class, IfExprTimestampScalarScalar.class,
 })
 @VectorizedExpressionsSupportDecimal64()
-public class GenericUDFIf extends GenericUDF {
+public class GenericUDFIf extends GenericUDF implements StatEstimatorProvider {
   private transient ObjectInspector[] argumentOIs;
   private transient GenericUDFUtils.ReturnObjectInspectorResolver returnOIResolver;
 
@@ -171,4 +178,22 @@ public class GenericUDFIf extends GenericUDF {
     assert (children.length == 3);
     return getStandardDisplayString("if", children);
   }
+
+  @Override
+  public StatEstimator getStatEstimator() {
+    return new IfStatEstimator();
+  }
+
+  static class IfStatEstimator implements StatEstimator {
+
+    @Override
+    public Optional<ColStatistics> estimate(List<ColStatistics> argStats) {
+      PessimisticStatCombiner combiner = new PessimisticStatCombiner();
+      combiner.add(argStats.get(1));
+      combiner.add(argStats.get(2));
+      return combiner.getResult();
+    }
+
+  }
+
 }
diff --git a/ql/src/test/queries/clientpositive/udf_coalesce.q b/ql/src/test/queries/clientpositive/udf_coalesce.q
index 7d87580..6c6594a 100644
--- a/ql/src/test/queries/clientpositive/udf_coalesce.q
+++ b/ql/src/test/queries/clientpositive/udf_coalesce.q
@@ -1,6 +1,7 @@
 --! qt:dataset:src_thrift
 --! qt:dataset:src
-set hive.fetch.task.conversion=more;
+set hive.cbo.enable=false;
+set hive.fetch.task.conversion=none;
 
 DESCRIBE FUNCTION coalesce;
 DESCRIBE FUNCTION EXTENDED coalesce;
@@ -47,6 +48,10 @@ SELECT COALESCE(1),
 FROM src tablesample (1 rows);
 
 EXPLAIN
+SELECT COALESCE(key,'x') from src limit 1;
+
+
+EXPLAIN
 SELECT COALESCE(src_thrift.lint[1], 999),
        COALESCE(src_thrift.lintstring[0].mystring, '999'),
        COALESCE(src_thrift.mstringstring['key_2'], '999')
diff --git a/ql/src/test/results/clientpositive/cbo_rp_gby2_map_multi_distinct.q.out b/ql/src/test/results/clientpositive/cbo_rp_gby2_map_multi_distinct.q.out
index 8579c83..c813456 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_gby2_map_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_gby2_map_multi_distinct.q.out
@@ -38,20 +38,20 @@ STAGE PLANS:
             Select Operator
               expressions: substr(key, 1, 1) (type: string), substr(value, 5) (type: string), value (type: string)
               outputColumnNames: $f0, $f1, $f2
-              Statistics: Num rows: 500 Data size: 229500 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 131000 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: count(DISTINCT $f1), sum($f1), sum(DISTINCT $f1), count($f2)
                 keys: $f0 (type: string), $f1 (type: string)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 250 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 50750 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 50750 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: double), _col5 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -59,14 +59,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: $f0, $f1, $f2, $f3, $f4
-          Statistics: Num rows: 250 Data size: 54000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 29250 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: $f0 (type: string), UDFToInteger($f1) (type: int), concat($f0, $f2) (type: string), UDFToInteger($f3) (type: int), UDFToInteger($f4) (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -75,7 +75,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int)
               outputColumnNames: key, c1, c2, c3, c4
-              Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(c1, 'hll'), compute_stats(c2, 'hll'), compute_stats(c3, 'hll'), compute_stats(c4, 'hll')
                 minReductionHashAggr: 0.99
@@ -204,20 +204,20 @@ STAGE PLANS:
             Select Operator
               expressions: substr(key, 1, 1) (type: string), substr(value, 5) (type: string), value (type: string)
               outputColumnNames: $f0, $f1, $f2
-              Statistics: Num rows: 500 Data size: 229500 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 131000 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: count(DISTINCT $f0), sum($f1), sum(DISTINCT $f1), count($f2)
                 keys: $f0 (type: string), $f1 (type: string)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 250 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 50750 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 50750 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: double), _col5 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -225,14 +225,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: $f0, $f1, $f2, $f3, $f4
-          Statistics: Num rows: 250 Data size: 54000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 29250 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: $f0 (type: string), UDFToInteger($f1) (type: int), concat($f0, $f2) (type: string), UDFToInteger($f3) (type: int), UDFToInteger($f4) (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -241,7 +241,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int)
               outputColumnNames: key, c1, c2, c3, c4
-              Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(c1, 'hll'), compute_stats(c2, 'hll'), compute_stats(c3, 'hll'), compute_stats(c4, 'hll')
                 minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/cbo_rp_groupby3_noskew_multi_distinct.q.out b/ql/src/test/results/clientpositive/cbo_rp_groupby3_noskew_multi_distinct.q.out
index 9ae6330..c1e3013 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_groupby3_noskew_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_groupby3_noskew_multi_distinct.q.out
@@ -55,12 +55,12 @@ STAGE PLANS:
             Select Operator
               expressions: substr(value, 5) (type: string), UDFToDouble(substr(value, 5)) (type: double), (UDFToDouble(substr(value, 5)) * UDFToDouble(substr(value, 5))) (type: double)
               outputColumnNames: $f0, $f00, $f2
-              Statistics: Num rows: 500 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 51000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: $f0 (type: string)
                 null sort order: z
                 sort order: +
-                Statistics: Num rows: 500 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 51000 Basic stats: COMPLETE Column stats: COMPLETE
                 value expressions: $f2 (type: double), $f00 (type: double)
       Execution mode: vectorized
       Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/constprog_when_case.q.out b/ql/src/test/results/clientpositive/constprog_when_case.q.out
index f75fa1c..8d3dd98 100644
--- a/ql/src/test/results/clientpositive/constprog_when_case.q.out
+++ b/ql/src/test/results/clientpositive/constprog_when_case.q.out
@@ -52,10 +52,10 @@ STAGE PLANS:
             Select Operator
               expressions: if((bool0 is true or (null and bool0 is not true and bool0 is not false)), key0, if((((not bool0) is true and bool0 is not true) or (null and bool0 is not true and bool0 is not false)), key1, key2)) (type: string)
               outputColumnNames: _col0
-              Statistics: Num rows: 1000 Data size: 184000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1000 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
-                Statistics: Num rows: 1000 Data size: 184000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/count_dist_rewrite.q.out b/ql/src/test/results/clientpositive/count_dist_rewrite.q.out
index f8dbec1..ee1cd28 100644
--- a/ql/src/test/results/clientpositive/count_dist_rewrite.q.out
+++ b/ql/src/test/results/clientpositive/count_dist_rewrite.q.out
@@ -762,13 +762,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6, _col7
-                Statistics: Num rows: 250 Data size: 146000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 121500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   null sort order: z
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 146000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 121500 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: double), _col2 (type: bigint), _col4 (type: string), _col5 (type: string), _col6 (type: double), _col7 (type: double)
       Reduce Operator Tree:
         Group By Operator
@@ -776,7 +776,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: partial2
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-          Statistics: Num rows: 250 Data size: 146000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 121500 Basic stats: COMPLETE Column stats: COMPLETE
           Group By Operator
             aggregations: sum(_col1), count(_col2), count(_col0), max(_col3), min(_col4), sum(_col5), sum(_col6)
             mode: partial2
diff --git a/ql/src/test/results/clientpositive/groupby11.q.out b/ql/src/test/results/clientpositive/groupby11.q.out
index a784779..4b62238 100644
--- a/ql/src/test/results/clientpositive/groupby11.q.out
+++ b/ql/src/test/results/clientpositive/groupby11.q.out
@@ -234,7 +234,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: partial1
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 500 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 500 Data size: 51000 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -251,7 +251,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 500 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 51000 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col1 (type: bigint), _col2 (type: bigint)
       Execution mode: vectorized
       Reduce Operator Tree:
@@ -260,14 +260,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: final
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 307 Data size: 61400 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 307 Data size: 31314 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 307 Data size: 58944 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 307 Data size: 28858 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 307 Data size: 58944 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 307 Data size: 28858 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -276,7 +276,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
               outputColumnNames: key, val1, val2
-              Statistics: Num rows: 307 Data size: 85653 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 307 Data size: 55567 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 table:
@@ -305,7 +305,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: rand() (type: double)
-              Statistics: Num rows: 307 Data size: 85653 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 307 Data size: 55567 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: key (type: string), val1 (type: int), val2 (type: int)
       Execution mode: vectorized
       Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/groupby2_map.q.out b/ql/src/test/results/clientpositive/groupby2_map.q.out
index 1569f04..85bebf0 100644
--- a/ql/src/test/results/clientpositive/groupby2_map.q.out
+++ b/ql/src/test/results/clientpositive/groupby2_map.q.out
@@ -41,13 +41,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 250 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 46750 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 46750 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: double)
       Reduce Operator Tree:
         Group By Operator
@@ -55,14 +55,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 250 Data size: 50000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 25250 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 250 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 68250 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 250 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 68250 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -71,7 +71,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string)
               outputColumnNames: key, c1, c2
-              Statistics: Num rows: 250 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 68250 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(c1, 'hll'), compute_stats(c2, 'hll')
                 minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/groupby2_map_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby2_map_multi_distinct.q.out
index 8d1b345..d49e7de 100644
--- a/ql/src/test/results/clientpositive/groupby2_map_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/groupby2_map_multi_distinct.q.out
@@ -41,13 +41,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 250 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 50750 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 50750 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: double), _col5 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -55,14 +55,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 250 Data size: 54000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 29250 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -71,7 +71,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int)
               outputColumnNames: key, c1, c2, c3, c4
-              Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(c1, 'hll'), compute_stats(c2, 'hll'), compute_stats(c3, 'hll'), compute_stats(c4, 'hll')
                 minReductionHashAggr: 0.99
@@ -195,13 +195,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 250 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 50750 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 50750 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: double), _col5 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -209,14 +209,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 250 Data size: 54000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 29250 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -225,7 +225,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int)
               outputColumnNames: key, c1, c2, c3, c4
-              Statistics: Num rows: 250 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(c1, 'hll'), compute_stats(c2, 'hll'), compute_stats(c3, 'hll'), compute_stats(c4, 'hll')
                 minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/groupby2_map_skew.q.out b/ql/src/test/results/clientpositive/groupby2_map_skew.q.out
index 762e286..9783f9e 100644
--- a/ql/src/test/results/clientpositive/groupby2_map_skew.q.out
+++ b/ql/src/test/results/clientpositive/groupby2_map_skew.q.out
@@ -42,13 +42,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 250 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 46750 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 46750 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: double)
       Reduce Operator Tree:
         Group By Operator
@@ -56,7 +56,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: partials
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 250 Data size: 50000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 25250 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -73,7 +73,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 250 Data size: 50000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 25250 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col1 (type: bigint), _col2 (type: double)
       Execution mode: vectorized
       Reduce Operator Tree:
@@ -82,14 +82,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: final
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 250 Data size: 50000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 25250 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 250 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 68250 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 250 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 68250 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -98,7 +98,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string)
               outputColumnNames: key, c1, c2
-              Statistics: Num rows: 250 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 68250 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(c1, 'hll'), compute_stats(c2, 'hll')
                 minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/groupby2_noskew.q.out b/ql/src/test/results/clientpositive/groupby2_noskew.q.out
index 6edcdc6..5dd501f 100644
--- a/ql/src/test/results/clientpositive/groupby2_noskew.q.out
+++ b/ql/src/test/results/clientpositive/groupby2_noskew.q.out
@@ -48,14 +48,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: complete
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 316 Data size: 63200 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 316 Data size: 31916 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -64,7 +64,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string)
               outputColumnNames: key, c1, c2
-              Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 table:
@@ -97,7 +97,7 @@ STAGE PLANS:
             Reduce Output Operator
               null sort order: 
               sort order: 
-              Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: key (type: string), c1 (type: int), c2 (type: string)
       Execution mode: vectorized
       Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/groupby2_noskew_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby2_noskew_multi_distinct.q.out
index ded98a4..4d8a5aa 100644
--- a/ql/src/test/results/clientpositive/groupby2_noskew_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/groupby2_noskew_multi_distinct.q.out
@@ -49,14 +49,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: complete
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 316 Data size: 68256 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 316 Data size: 36972 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -65,7 +65,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int)
               outputColumnNames: key, c1, c2, c3, c4
-              Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 table:
@@ -98,7 +98,7 @@ STAGE PLANS:
             Reduce Output Operator
               null sort order: 
               sort order: 
-              Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: key (type: string), c1 (type: int), c2 (type: string), c3 (type: int), c4 (type: int)
       Execution mode: vectorized
       Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/groupby3_map.q.out b/ql/src/test/results/clientpositive/groupby3_map.q.out
index af02802..22573cb 100644
--- a/ql/src/test/results/clientpositive/groupby3_map.q.out
+++ b/ql/src/test/results/clientpositive/groupby3_map.q.out
@@ -58,12 +58,12 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 250 Data size: 150000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 125500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   null sort order: z
                   sort order: +
-                  Statistics: Num rows: 250 Data size: 150000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 125500 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: double), _col2 (type: bigint), _col5 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: double)
       Reduce Operator Tree:
         Group By Operator
diff --git a/ql/src/test/results/clientpositive/groupby3_map_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby3_map_multi_distinct.q.out
index 1f56188..fbd6e4e 100644
--- a/ql/src/test/results/clientpositive/groupby3_map_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/groupby3_map_multi_distinct.q.out
@@ -62,12 +62,12 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 250 Data size: 150000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 125500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   null sort order: z
                   sort order: +
-                  Statistics: Num rows: 250 Data size: 150000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 125500 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: double), _col2 (type: bigint), _col5 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: double)
       Reduce Operator Tree:
         Group By Operator
diff --git a/ql/src/test/results/clientpositive/groupby3_map_skew.q.out b/ql/src/test/results/clientpositive/groupby3_map_skew.q.out
index f315663..c6cccd1 100644
--- a/ql/src/test/results/clientpositive/groupby3_map_skew.q.out
+++ b/ql/src/test/results/clientpositive/groupby3_map_skew.q.out
@@ -59,13 +59,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-                Statistics: Num rows: 250 Data size: 348000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 323500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   null sort order: z
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 348000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 323500 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: double), _col2 (type: struct<count:bigint,sum:double,input:string>), _col4 (type: string), _col5 (type: string), _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,variance:double>), _col9 (type: struct<count:bigint,sum:double,variance:double>)
       Reduce Operator Tree:
         Group By Operator
diff --git a/ql/src/test/results/clientpositive/groupby4.q.out b/ql/src/test/results/clientpositive/groupby4.q.out
index 7528034..d861e47 100644
--- a/ql/src/test/results/clientpositive/groupby4.q.out
+++ b/ql/src/test/results/clientpositive/groupby4.q.out
@@ -49,7 +49,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: partial1
           outputColumnNames: _col0
-          Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -66,17 +66,17 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)
           mode: final
           outputColumnNames: _col0
-          Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -85,7 +85,7 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: string)
             outputColumnNames: c1
-            Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -119,7 +119,7 @@ STAGE PLANS:
               null sort order: 
               sort order: 
               Map-reduce partition columns: rand() (type: double)
-              Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: c1 (type: string)
       Execution mode: vectorized
       Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/groupby4_noskew.q.out b/ql/src/test/results/clientpositive/groupby4_noskew.q.out
index 6550017..842bda3 100644
--- a/ql/src/test/results/clientpositive/groupby4_noskew.q.out
+++ b/ql/src/test/results/clientpositive/groupby4_noskew.q.out
@@ -47,10 +47,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: complete
           outputColumnNames: _col0
-          Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -59,7 +59,7 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: string)
             outputColumnNames: c1
-            Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -92,7 +92,7 @@ STAGE PLANS:
             Reduce Output Operator
               null sort order: 
               sort order: 
-              Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: c1 (type: string)
       Execution mode: vectorized
       Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/groupby6.q.out b/ql/src/test/results/clientpositive/groupby6.q.out
index be673c5..a72afb2 100644
--- a/ql/src/test/results/clientpositive/groupby6.q.out
+++ b/ql/src/test/results/clientpositive/groupby6.q.out
@@ -49,7 +49,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: partial1
           outputColumnNames: _col0
-          Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -66,17 +66,17 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)
           mode: final
           outputColumnNames: _col0
-          Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -85,7 +85,7 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: string)
             outputColumnNames: c1
-            Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -119,7 +119,7 @@ STAGE PLANS:
               null sort order: 
               sort order: 
               Map-reduce partition columns: rand() (type: double)
-              Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: c1 (type: string)
       Execution mode: vectorized
       Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/groupby6_map.q.out b/ql/src/test/results/clientpositive/groupby6_map.q.out
index ed812ce..9f8c791 100644
--- a/ql/src/test/results/clientpositive/groupby6_map.q.out
+++ b/ql/src/test/results/clientpositive/groupby6_map.q.out
@@ -40,23 +40,23 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   null sort order: z
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -65,7 +65,7 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: string)
             outputColumnNames: c1
-            Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
               aggregations: compute_stats(c1, 'hll')
               minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/groupby6_map_skew.q.out b/ql/src/test/results/clientpositive/groupby6_map_skew.q.out
index 57f8c3f..43fbc84 100644
--- a/ql/src/test/results/clientpositive/groupby6_map_skew.q.out
+++ b/ql/src/test/results/clientpositive/groupby6_map_skew.q.out
@@ -41,20 +41,20 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   null sort order: z
                   sort order: +
                   Map-reduce partition columns: rand() (type: double)
-                  Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)
           mode: partials
           outputColumnNames: _col0
-          Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -71,17 +71,17 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)
           mode: final
           outputColumnNames: _col0
-          Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -90,7 +90,7 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: string)
             outputColumnNames: c1
-            Statistics: Num rows: 250 Data size: 46000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 250 Data size: 21250 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
               aggregations: compute_stats(c1, 'hll')
               minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/groupby6_noskew.q.out b/ql/src/test/results/clientpositive/groupby6_noskew.q.out
index 689775e..9545ca3 100644
--- a/ql/src/test/results/clientpositive/groupby6_noskew.q.out
+++ b/ql/src/test/results/clientpositive/groupby6_noskew.q.out
@@ -47,10 +47,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: complete
           outputColumnNames: _col0
-          Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -59,7 +59,7 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: string)
             outputColumnNames: c1
-            Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -92,7 +92,7 @@ STAGE PLANS:
             Reduce Output Operator
               null sort order: 
               sort order: 
-              Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: c1 (type: string)
       Execution mode: vectorized
       Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/groupby8_map_skew.q.out b/ql/src/test/results/clientpositive/groupby8_map_skew.q.out
index 8191963..66719b3 100644
--- a/ql/src/test/results/clientpositive/groupby8_map_skew.q.out
+++ b/ql/src/test/results/clientpositive/groupby8_map_skew.q.out
@@ -59,13 +59,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: key (type: string), value (type: string)
               outputColumnNames: key, value
@@ -76,7 +76,7 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -204,7 +204,7 @@ STAGE PLANS:
               null sort order: zz
               sort order: ++
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
diff --git a/ql/src/test/results/clientpositive/groupby9.q.out b/ql/src/test/results/clientpositive/groupby9.q.out
index d46ca78..0170f97 100644
--- a/ql/src/test/results/clientpositive/groupby9.q.out
+++ b/ql/src/test/results/clientpositive/groupby9.q.out
@@ -57,13 +57,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: key (type: string), value (type: string)
               outputColumnNames: key, value
@@ -74,7 +74,7 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 500 Data size: 185000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 136000 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -176,7 +176,7 @@ STAGE PLANS:
               null sort order: zzz
               sort order: +++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-              Statistics: Num rows: 500 Data size: 185000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 136000 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
@@ -944,13 +944,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: key (type: string), value (type: string)
               outputColumnNames: key, value
@@ -961,7 +961,7 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 500 Data size: 185000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 136000 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -1063,7 +1063,7 @@ STAGE PLANS:
               null sort order: zzz
               sort order: +++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-              Statistics: Num rows: 500 Data size: 185000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 136000 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
@@ -1831,13 +1831,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: key (type: string), value (type: string)
               outputColumnNames: key, value
@@ -1848,7 +1848,7 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 500 Data size: 185000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 136000 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -1950,7 +1950,7 @@ STAGE PLANS:
               null sort order: zzz
               sort order: +++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-              Statistics: Num rows: 500 Data size: 185000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 136000 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
@@ -3608,13 +3608,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 45250 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: key (type: string), value (type: string)
               outputColumnNames: key, value
@@ -3625,7 +3625,7 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 500 Data size: 185000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 136000 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -3727,7 +3727,7 @@ STAGE PLANS:
               null sort order: zzz
               sort order: +++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-              Statistics: Num rows: 500 Data size: 185000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 136000 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
diff --git a/ql/src/test/results/clientpositive/groupby_map_ppr.q.out b/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
index afdb705..952f310 100644
--- a/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
+++ b/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
@@ -57,13 +57,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 93500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 93500 Basic stats: COMPLETE Column stats: COMPLETE
                   tag: -1
                   value expressions: _col3 (type: double)
                   auto parallelism: false
@@ -178,17 +178,17 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 316 Data size: 63200 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 316 Data size: 31916 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
@@ -219,7 +219,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string)
               outputColumnNames: key, c1, c2
-              Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(c1, 'hll'), compute_stats(c2, 'hll')
                 minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
index a0a0ea4..bd43f54 100644
--- a/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
@@ -57,13 +57,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                Statistics: Num rows: 1000 Data size: 491000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 294000 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                   null sort order: zzz
                   sort order: +++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 1000 Data size: 491000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1000 Data size: 294000 Basic stats: COMPLETE Column stats: COMPLETE
                   tag: -1
                   value expressions: _col4 (type: double)
                   auto parallelism: false
@@ -178,17 +178,17 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 316 Data size: 68256 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 316 Data size: 36972 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
@@ -219,7 +219,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int)
               outputColumnNames: key, c1, c2, c3, c4
-              Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(c1, 'hll'), compute_stats(c2, 'hll'), compute_stats(c3, 'hll'), compute_stats(c4, 'hll')
                 minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/groupby_multi_single_reducer.q.out b/ql/src/test/results/clientpositive/groupby_multi_single_reducer.q.out
index 8fa21f2..756c179 100644
--- a/ql/src/test/results/clientpositive/groupby_multi_single_reducer.q.out
+++ b/ql/src/test/results/clientpositive/groupby_multi_single_reducer.q.out
@@ -94,14 +94,14 @@ STAGE PLANS:
             keys: KEY._col0 (type: string)
             mode: complete
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            Statistics: Num rows: 316 Data size: 68256 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 36972 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
-              Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
-                Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -109,20 +109,20 @@ STAGE PLANS:
                     name: default.dest_g4
           Filter Operator
             predicate: (KEY._col0 >= 5) (type: boolean)
-            Statistics: Num rows: 166 Data size: 45650 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 166 Data size: 29216 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
               aggregations: count(DISTINCT KEY._col1:0._col0), sum(KEY._col1:0._col0), sum(DISTINCT KEY._col1:1._col0), count(VALUE._col0)
               keys: KEY._col0 (type: string)
               mode: complete
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
-              Statistics: Num rows: 105 Data size: 22680 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 105 Data size: 12285 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 105 Data size: 39900 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 105 Data size: 29505 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 105 Data size: 39900 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 105 Data size: 29505 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -130,20 +130,20 @@ STAGE PLANS:
                       name: default.dest_g2
           Filter Operator
             predicate: (KEY._col0 < 5) (type: boolean)
-            Statistics: Num rows: 166 Data size: 45650 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 166 Data size: 29216 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
               aggregations: count(DISTINCT KEY._col1:0._col0), sum(KEY._col1:0._col0), sum(DISTINCT KEY._col1:1._col0), count(VALUE._col0)
               keys: KEY._col0 (type: string)
               mode: complete
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
-              Statistics: Num rows: 105 Data size: 22680 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 105 Data size: 12285 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 105 Data size: 39900 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 105 Data size: 29505 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 105 Data size: 39900 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 105 Data size: 29505 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -349,14 +349,14 @@ STAGE PLANS:
             keys: KEY._col0 (type: string)
             mode: complete
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            Statistics: Num rows: 316 Data size: 68256 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 36972 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
-              Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
-                Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -364,20 +364,20 @@ STAGE PLANS:
                     name: default.dest_g4
           Filter Operator
             predicate: (KEY._col0 >= 5) (type: boolean)
-            Statistics: Num rows: 166 Data size: 45650 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 166 Data size: 29216 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
               aggregations: count(DISTINCT KEY._col1:0._col0), sum(KEY._col1:0._col0), sum(DISTINCT KEY._col1:1._col0), count(VALUE._col0)
               keys: KEY._col0 (type: string)
               mode: complete
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
-              Statistics: Num rows: 105 Data size: 22680 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 105 Data size: 12285 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 105 Data size: 39900 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 105 Data size: 29505 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 105 Data size: 39900 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 105 Data size: 29505 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -385,20 +385,20 @@ STAGE PLANS:
                       name: default.dest_g2
           Filter Operator
             predicate: (KEY._col0 < 5) (type: boolean)
-            Statistics: Num rows: 166 Data size: 45650 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 166 Data size: 29216 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
               aggregations: count(DISTINCT KEY._col1:0._col0), sum(KEY._col1:0._col0), sum(DISTINCT KEY._col1:1._col0), count(VALUE._col0)
               keys: KEY._col0 (type: string)
               mode: complete
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
-              Statistics: Num rows: 105 Data size: 22680 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 105 Data size: 12285 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 105 Data size: 39900 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 105 Data size: 29505 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 105 Data size: 39900 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 105 Data size: 29505 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -467,11 +467,11 @@ STAGE PLANS:
             keys: KEY._col0 (type: string), KEY._col1 (type: string)
             mode: complete
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            Statistics: Num rows: 500 Data size: 196000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 97000 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: _col0 (type: string), _col2 (type: bigint), concat(_col0, _col3) (type: string), _col3 (type: double), _col4 (type: bigint)
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
-              Statistics: Num rows: 500 Data size: 196000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 146500 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 table:
@@ -480,20 +480,20 @@ STAGE PLANS:
                     serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
           Filter Operator
             predicate: (KEY._col0 >= 5) (type: boolean)
-            Statistics: Num rows: 166 Data size: 76194 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 166 Data size: 43326 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
               aggregations: count(DISTINCT KEY._col2:0._col0), sum(KEY._col2:0._col0), count(VALUE._col0)
               keys: KEY._col0 (type: string), KEY._col1 (type: string)
               mode: complete
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
-              Statistics: Num rows: 166 Data size: 65072 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 166 Data size: 32204 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: _col0 (type: string), UDFToInteger(_col2) (type: int), concat(_col0, _col3) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 166 Data size: 63080 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 166 Data size: 46646 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 166 Data size: 63080 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 166 Data size: 46646 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -508,7 +508,7 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: bigint)
               null sort order: zz
               sort order: ++
-              Statistics: Num rows: 500 Data size: 196000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 146500 Basic stats: COMPLETE Column stats: COMPLETE
               TopN Hash Memory Usage: 0.1
               value expressions: _col2 (type: string), _col3 (type: double), _col4 (type: bigint)
       Execution mode: vectorized
@@ -516,17 +516,17 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: bigint), VALUE._col0 (type: string), VALUE._col1 (type: double), VALUE._col2 (type: bigint)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 500 Data size: 196000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 500 Data size: 146500 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 10
-            Statistics: Num rows: 10 Data size: 3920 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 2930 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
-              Statistics: Num rows: 10 Data size: 3800 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 10 Data size: 2810 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
-                Statistics: Num rows: 10 Data size: 3800 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 10 Data size: 2810 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
diff --git a/ql/src/test/results/clientpositive/groupby_multi_single_reducer2.q.out b/ql/src/test/results/clientpositive/groupby_multi_single_reducer2.q.out
index e4bc26e..d151470 100644
--- a/ql/src/test/results/clientpositive/groupby_multi_single_reducer2.q.out
+++ b/ql/src/test/results/clientpositive/groupby_multi_single_reducer2.q.out
@@ -63,20 +63,20 @@ STAGE PLANS:
           Statistics: Num rows: 332 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
           Filter Operator
             predicate: (KEY._col0 >= 5) (type: boolean)
-            Statistics: Num rows: 110 Data size: 30250 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 110 Data size: 19360 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
               aggregations: count(DISTINCT KEY._col1:0._col0)
               keys: KEY._col0 (type: string)
               mode: complete
               outputColumnNames: _col0, _col1
-              Statistics: Num rows: 105 Data size: 20160 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 105 Data size: 9765 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: _col0 (type: string), UDFToInteger(_col1) (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 105 Data size: 19740 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 105 Data size: 9345 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 105 Data size: 19740 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 105 Data size: 9345 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -85,7 +85,7 @@ STAGE PLANS:
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: int)
                   outputColumnNames: key, c1
-                  Statistics: Num rows: 105 Data size: 19740 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 105 Data size: 9345 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: compute_stats(key, 'hll'), compute_stats(c1, 'hll')
                     minReductionHashAggr: 0.99
@@ -100,20 +100,20 @@ STAGE PLANS:
                           serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
           Filter Operator
             predicate: (KEY._col0 < 5) (type: boolean)
-            Statistics: Num rows: 110 Data size: 30250 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 110 Data size: 19360 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
               aggregations: count(DISTINCT KEY._col1:0._col0), count(VALUE._col0)
               keys: KEY._col0 (type: string)
               mode: complete
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 105 Data size: 21000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 105 Data size: 10605 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 105 Data size: 20160 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 105 Data size: 9765 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 105 Data size: 20160 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 105 Data size: 9765 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -122,7 +122,7 @@ STAGE PLANS:
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
                   outputColumnNames: key, c1, c2
-                  Statistics: Num rows: 105 Data size: 20160 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 105 Data size: 9765 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: compute_stats(key, 'hll'), compute_stats(c1, 'hll'), compute_stats(c2, 'hll')
                     minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/groupby_nocolumnalign.q.out b/ql/src/test/results/clientpositive/groupby_nocolumnalign.q.out
index 19ae138..46d34fb 100644
--- a/ql/src/test/results/clientpositive/groupby_nocolumnalign.q.out
+++ b/ql/src/test/results/clientpositive/groupby_nocolumnalign.q.out
@@ -41,13 +41,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 500 Data size: 185000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 136000 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                   null sort order: zzz
                   sort order: +++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                  Statistics: Num rows: 500 Data size: 185000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 136000 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(DISTINCT KEY._col2:0._col0)
diff --git a/ql/src/test/results/clientpositive/groupby_position.q.out b/ql/src/test/results/clientpositive/groupby_position.q.out
index f52623a..af0c243 100644
--- a/ql/src/test/results/clientpositive/groupby_position.q.out
+++ b/ql/src/test/results/clientpositive/groupby_position.q.out
@@ -56,13 +56,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 83 Data size: 23157 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 83 Data size: 15023 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 83 Data size: 23157 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 83 Data size: 15023 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 20) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
@@ -72,7 +72,7 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 166 Data size: 61420 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 83 Data size: 22576 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -174,7 +174,7 @@ STAGE PLANS:
               null sort order: zzz
               sort order: +++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-              Statistics: Num rows: 166 Data size: 61420 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 83 Data size: 22576 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
@@ -182,14 +182,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 166 Data size: 30876 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 83 Data size: 15438 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: UDFToInteger(_col0) (type: int), _col1 (type: string), CAST( _col2 AS STRING) (type: string)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 166 Data size: 46314 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 83 Data size: 23157 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 166 Data size: 46314 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 83 Data size: 23157 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -198,7 +198,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
               outputColumnNames: key, val1, val2
-              Statistics: Num rows: 166 Data size: 46314 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 83 Data size: 23157 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(val1, 'hll'), compute_stats(val2, 'hll')
                 minReductionHashAggr: 0.99
@@ -349,13 +349,13 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 83 Data size: 23157 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 83 Data size: 15023 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 83 Data size: 23157 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 83 Data size: 15023 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 20) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
@@ -365,7 +365,7 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 166 Data size: 61420 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 83 Data size: 22576 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -467,7 +467,7 @@ STAGE PLANS:
               null sort order: zzz
               sort order: +++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-              Statistics: Num rows: 166 Data size: 61420 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 83 Data size: 22576 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
@@ -475,14 +475,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 166 Data size: 30876 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 83 Data size: 15438 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: UDFToInteger(_col1) (type: int), _col0 (type: string), CAST( _col2 AS STRING) (type: string)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 166 Data size: 46314 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 83 Data size: 23157 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 166 Data size: 46314 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 83 Data size: 23157 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -491,7 +491,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
               outputColumnNames: key, val1, val2
-              Statistics: Num rows: 166 Data size: 46314 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 83 Data size: 23157 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(val1, 'hll'), compute_stats(val2, 'hll')
                 minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/groupby_ppr.q.out b/ql/src/test/results/clientpositive/groupby_ppr.q.out
index 085ac2c..d7549d9 100644
--- a/ql/src/test/results/clientpositive/groupby_ppr.q.out
+++ b/ql/src/test/results/clientpositive/groupby_ppr.q.out
@@ -171,17 +171,17 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: complete
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 316 Data size: 63200 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 316 Data size: 31916 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
@@ -212,7 +212,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string)
               outputColumnNames: key, c1, c2
-              Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 GlobalTableId: 0
@@ -279,7 +279,7 @@ STAGE PLANS:
             Reduce Output Operator
               null sort order: 
               sort order: 
-              Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
               value expressions: key (type: string), c1 (type: int), c2 (type: string)
               auto parallelism: false
diff --git a/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
index 0a7275f..95f95b0 100644
--- a/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
@@ -171,17 +171,17 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: complete
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 316 Data size: 68256 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 316 Data size: 36972 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string), UDFToInteger(_col3) (type: int), UDFToInteger(_col4) (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
@@ -212,7 +212,7 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int)
               outputColumnNames: key, c1, c2, c3, c4
-              Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 GlobalTableId: 0
@@ -279,7 +279,7 @@ STAGE PLANS:
             Reduce Output Operator
               null sort order: 
               sort order: 
-              Statistics: Num rows: 316 Data size: 120080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
               value expressions: key (type: string), c1 (type: int), c2 (type: string), c3 (type: int), c4 (type: int)
               auto parallelism: false
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 ed969fa..9b869a1 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
@@ -419,15 +419,14 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-4 depends on stages: Stage-0, Stage-2
-  Stage-3 depends on stages: Stage-1
-  Stage-9 depends on stages: Stage-3 , consists of Stage-6, Stage-5, Stage-7
-  Stage-6
-  Stage-0 depends on stages: Stage-6, Stage-5, Stage-8
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
+  Stage-4
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+  Stage-2 depends on stages: Stage-0, Stage-8
+  Stage-3
   Stage-5
-  Stage-7
-  Stage-8 depends on stages: Stage-7
+  Stage-6 depends on stages: Stage-5
+  Stage-8 depends on stages: Stage-1
 
 STAGE PLANS:
   Stage: Stage-1
@@ -466,100 +465,37 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: string), CAST( _col1 AS STRING) (type: string), if(((UDFToDouble(_col0) % 100.0D) = 0.0D), '11', '12') (type: string)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 316 Data size: 143780 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 112812 Basic stats: COMPLETE Column stats: COMPLETE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 316 Data size: 112812 Basic stats: COMPLETE Column stats: COMPLETE
+              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.test_table_n8
             Select Operator
               expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col2 (type: string)
               outputColumnNames: key, value, ds, hr
-              Statistics: Num rows: 316 Data size: 173484 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 142516 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                 keys: ds (type: string), hr (type: string)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 158 Data size: 182964 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                 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
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col0 (type: string), _col1 (type: string)
-              null sort order: zz
-              sort order: ++
-              Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-              Statistics: Num rows: 158 Data size: 182964 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
-      Execution mode: vectorized
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1)
-          keys: KEY._col0 (type: string), KEY._col1 (type: string)
-          mode: mergepartial
-          outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 158 Data size: 182964 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: string), _col1 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 158 Data size: 182964 Basic stats: COMPLETE Column stats: COMPLETE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 158 Data size: 182964 Basic stats: COMPLETE Column stats: COMPLETE
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
-  Stage: Stage-4
-    Stats Work
-      Basic Stats Work:
-      Column Stats Desc:
-          Columns: key, value
-          Column Types: string, string
-          Table: default.test_table_n8
 
-  Stage: Stage-3
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col2 (type: string)
-              null sort order: a
-              sort order: +
-              Map-reduce partition columns: _col2 (type: string)
-              Statistics: Num rows: 316 Data size: 143780 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col0 (type: string), _col1 (type: string)
-      Execution mode: vectorized
-      Reduce Operator Tree:
-        Select Operator
-          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), KEY._col2 (type: string)
-          outputColumnNames: _col0, _col1, _col2
-          File Output Operator
-            compressed: false
-            Dp Sort State: PARTITION_SORTED
-            Statistics: Num rows: 316 Data size: 143780 Basic stats: COMPLETE Column stats: COMPLETE
-            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.test_table_n8
-
-  Stage: Stage-9
+  Stage: Stage-7
     Conditional Operator
 
-  Stage: Stage-6
+  Stage: Stage-4
     Move Operator
       files:
           hdfs directory: true
@@ -578,26 +514,65 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
               name: default.test_table_n8
 
-  Stage: Stage-5
+  Stage: Stage-2
+    Stats Work
+      Basic Stats Work:
+      Column Stats Desc:
+          Columns: key, value
+          Column Types: string, string
+          Table: default.test_table_n8
+
+  Stage: Stage-3
     Merge File Operator
       Map Operator Tree:
           RCFile Merge Operator
       merge level: block
       input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
 
-  Stage: Stage-7
+  Stage: Stage-5
     Merge File Operator
       Map Operator Tree:
           RCFile Merge Operator
       merge level: block
       input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
 
-  Stage: Stage-8
+  Stage: Stage-6
     Move Operator
       files:
           hdfs directory: true
 #### A masked pattern was here ####
 
+  Stage: Stage-8
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: string), _col1 (type: string)
+              null sort order: zz
+              sort order: ++
+              Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+              Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1)
+          keys: KEY._col0 (type: string), KEY._col1 (type: string)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: string), _col1 (type: string)
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
 PREHOOK: query: INSERT OVERWRITE TABLE test_table_n8 PARTITION (ds = '2008-04-08', hr)
 SELECT key, value, IF (key % 100 == 0, '11', '12') FROM
 (SELECT key, COUNT(*) AS value FROM srcpart
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.out b/ql/src/test/results/clientpositive/list_bucket_dml_6.q.out
index f88f5bf..3ec1d32 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_6.q.out
@@ -49,14 +49,14 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), value (type: string), if(((UDFToDouble(key) % 100.0D) = 0.0D), 'a1', 'b1') (type: string)
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 GlobalTableId: 1
 #### A masked pattern was here ####
                 NumFilesPerFileSink: 1
                 Static Partition Specification: ds=2008-04-08/
-                Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
                 table:
                     input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
@@ -84,20 +84,20 @@ STAGE PLANS:
               Select Operator
                 expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col2 (type: string)
                 outputColumnNames: key, value, ds, hr
-                Statistics: Num rows: 1000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 358000 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                   keys: ds (type: string), hr (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                     tag: -1
                     value expressions: _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
                     auto parallelism: false
@@ -212,17 +212,17 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: string), _col1 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -430,14 +430,14 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), value (type: string), if(((UDFToDouble(key) % 100.0D) = 0.0D), 'a1', 'b1') (type: string)
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 GlobalTableId: 1
 #### A masked pattern was here ####
                 NumFilesPerFileSink: 1
                 Static Partition Specification: ds=2008-04-08/
-                Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
                 table:
                     input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
@@ -465,20 +465,20 @@ STAGE PLANS:
               Select Operator
                 expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col2 (type: string)
                 outputColumnNames: key, value, ds, hr
-                Statistics: Num rows: 1000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 358000 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                   keys: ds (type: string), hr (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                     tag: -1
                     value expressions: _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
                     auto parallelism: false
@@ -593,17 +593,17 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: string), _col1 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
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 43804d6..c41ae71 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
@@ -49,14 +49,14 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), value (type: string), if(((UDFToDouble(key) % 100.0D) = 0.0D), 'a1', 'b1') (type: string)
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 GlobalTableId: 1
 #### A masked pattern was here ####
                 NumFilesPerFileSink: 1
                 Static Partition Specification: ds=2008-04-08/
-                Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
                 table:
                     input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
@@ -84,20 +84,20 @@ STAGE PLANS:
               Select Operator
                 expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col2 (type: string)
                 outputColumnNames: key, value, ds, hr
-                Statistics: Num rows: 1000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 358000 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                   keys: ds (type: string), hr (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                     tag: -1
                     value expressions: _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
                     auto parallelism: false
@@ -212,17 +212,17 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: string), _col1 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -430,14 +430,14 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), value (type: string), if(((UDFToDouble(key) % 100.0D) = 0.0D), 'a1', 'b1') (type: string)
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 GlobalTableId: 1
 #### A masked pattern was here ####
                 NumFilesPerFileSink: 1
                 Static Partition Specification: ds=2008-04-08/
-                Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
                 table:
                     input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
@@ -465,20 +465,20 @@ STAGE PLANS:
               Select Operator
                 expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col2 (type: string)
                 outputColumnNames: key, value, ds, hr
-                Statistics: Num rows: 1000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 358000 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                   keys: ds (type: string), hr (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                     tag: -1
                     value expressions: _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
                     auto parallelism: false
@@ -593,17 +593,17 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: string), _col1 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_8.q.out b/ql/src/test/results/clientpositive/list_bucket_dml_8.q.out
index 8551f57..2b82c86 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_8.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_8.q.out
@@ -49,14 +49,14 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), value (type: string), if(((UDFToDouble(key) % 100.0D) = 0.0D), 'a1', 'b1') (type: string)
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 GlobalTableId: 1
 #### A masked pattern was here ####
                 NumFilesPerFileSink: 1
                 Static Partition Specification: ds=2008-04-08/
-                Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
                 table:
                     input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
@@ -84,20 +84,20 @@ STAGE PLANS:
               Select Operator
                 expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col2 (type: string)
                 outputColumnNames: key, value, ds, hr
-                Statistics: Num rows: 1000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 358000 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                   keys: ds (type: string), hr (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                     tag: -1
                     value expressions: _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
                     auto parallelism: false
@@ -212,17 +212,17 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: string), _col1 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
diff --git a/ql/src/test/results/clientpositive/llap/count_dist_rewrite.q.out b/ql/src/test/results/clientpositive/llap/count_dist_rewrite.q.out
index 2276b99..81069fd 100644
--- a/ql/src/test/results/clientpositive/llap/count_dist_rewrite.q.out
+++ b/ql/src/test/results/clientpositive/llap/count_dist_rewrite.q.out
@@ -763,13 +763,13 @@ STAGE PLANS:
                       minReductionHashAggr: 0.5
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6, _col7
-                      Statistics: Num rows: 250 Data size: 146000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 121500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         null sort order: z
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 250 Data size: 146000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 250 Data size: 121500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: double), _col2 (type: bigint), _col4 (type: string), _col5 (type: string), _col6 (type: double), _col7 (type: double)
             Execution mode: llap
             LLAP IO: no inputs
@@ -781,7 +781,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: partial2
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                Statistics: Num rows: 250 Data size: 146000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 121500 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col1), count(_col2), count(_col0), max(_col3), min(_col4), sum(_col5), sum(_col6)
                   mode: partial2
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_2.q.out b/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
index 738bf44..3613e12 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
@@ -3564,7 +3564,7 @@ Stage-4
                       <-Reducer 4 [SIMPLE_EDGE] llap
                         PARTITION_ONLY_SHUFFLE [RS_17]
                           PartitionCols:_col0
-                          Group By Operator [GBY_16] (rows=1 width=280)
+                          Group By Operator [GBY_16] (rows=1 width=275)
                             Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                             Group By Operator [GBY_13] (rows=1 width=272)
                               Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
@@ -3612,7 +3612,7 @@ Stage-4
                         Output:["key","val1","val2"]
                         Select Operator [SEL_33] (rows=1 width=456)
                           Output:["_col0","_col1","_col2"]
-                          Group By Operator [GBY_32] (rows=1 width=464)
+                          Group By Operator [GBY_32] (rows=1 width=459)
                             Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1
                              Please refer to the previous Group By Operator [GBY_13]
 Stage-5
@@ -3787,7 +3787,7 @@ Stage-4
                         <-Map 8 [CONTAINS] llap
                           Reduce Output Operator [RS_52]
                             PartitionCols:_col0
-                            Group By Operator [GBY_50] (rows=1 width=280)
+                            Group By Operator [GBY_50] (rows=1 width=275)
                               Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                               Select Operator [SEL_48] (rows=500 width=10)
                                 Output:["_col0","_col1"]
@@ -3795,13 +3795,13 @@ Stage-4
                                   Output:["key","value"]
                           Reduce Output Operator [RS_53]
                             PartitionCols:_col0, _col1
-                            Group By Operator [GBY_51] (rows=1 width=464)
+                            Group By Operator [GBY_51] (rows=1 width=459)
                               Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
                                Please refer to the previous Select Operator [SEL_48]
                         <-Map 9 [CONTAINS] llap
                           Reduce Output Operator [RS_59]
                             PartitionCols:_col0
-                            Group By Operator [GBY_57] (rows=1 width=280)
+                            Group By Operator [GBY_57] (rows=1 width=275)
                               Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                               Select Operator [SEL_55] (rows=500 width=10)
                                 Output:["_col0","_col1"]
@@ -3809,13 +3809,13 @@ Stage-4
                                   Output:["key","value"]
                           Reduce Output Operator [RS_60]
                             PartitionCols:_col0, _col1
-                            Group By Operator [GBY_58] (rows=1 width=464)
+                            Group By Operator [GBY_58] (rows=1 width=459)
                               Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
                                Please refer to the previous Select Operator [SEL_55]
                         <-Reducer 2 [CONTAINS] llap
                           Reduce Output Operator [RS_45]
                             PartitionCols:_col0
-                            Group By Operator [GBY_43] (rows=1 width=280)
+                            Group By Operator [GBY_43] (rows=1 width=275)
                               Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                               Select Operator [SEL_41] (rows=1 width=272)
                                 Output:["_col0","_col1"]
@@ -3830,7 +3830,7 @@ Stage-4
                                           default@src,s1,Tbl:COMPLETE,Col:COMPLETE
                           Reduce Output Operator [RS_46]
                             PartitionCols:_col0, _col1
-                            Group By Operator [GBY_44] (rows=1 width=464)
+                            Group By Operator [GBY_44] (rows=1 width=459)
                               Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
                                Please refer to the previous Select Operator [SEL_41]
                   PARTITION_ONLY_SHUFFLE [RS_22]
@@ -3923,7 +3923,7 @@ Stage-4
                         <-Map 8 [CONTAINS] llap
                           Reduce Output Operator [RS_50]
                             PartitionCols:_col0
-                            Group By Operator [GBY_48] (rows=1 width=280)
+                            Group By Operator [GBY_48] (rows=1 width=275)
                               Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                               Select Operator [SEL_46] (rows=500 width=10)
                                 Output:["_col0","_col1"]
@@ -3931,13 +3931,13 @@ Stage-4
                                   Output:["key","value"]
                           Reduce Output Operator [RS_51]
                             PartitionCols:_col0, _col1
-                            Group By Operator [GBY_49] (rows=1 width=464)
+                            Group By Operator [GBY_49] (rows=1 width=459)
                               Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
                                Please refer to the previous Select Operator [SEL_46]
                         <-Reducer 2 [CONTAINS] llap
                           Reduce Output Operator [RS_43]
                             PartitionCols:_col0
-                            Group By Operator [GBY_41] (rows=1 width=280)
+                            Group By Operator [GBY_41] (rows=1 width=275)
                               Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                               Select Operator [SEL_39] (rows=1 width=272)
                                 Output:["_col0","_col1"]
@@ -3952,7 +3952,7 @@ Stage-4
                                           default@src,s1,Tbl:COMPLETE,Col:COMPLETE
                           Reduce Output Operator [RS_44]
                             PartitionCols:_col0, _col1
-                            Group By Operator [GBY_42] (rows=1 width=464)
+                            Group By Operator [GBY_42] (rows=1 width=459)
                               Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
                                Please refer to the previous Select Operator [SEL_39]
                   PARTITION_ONLY_SHUFFLE [RS_20]
diff --git a/ql/src/test/results/clientpositive/llap/groupby2.q.out b/ql/src/test/results/clientpositive/llap/groupby2.q.out
index 848b503..bdad787 100644
--- a/ql/src/test/results/clientpositive/llap/groupby2.q.out
+++ b/ql/src/test/results/clientpositive/llap/groupby2.q.out
@@ -60,13 +60,13 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: partial1
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 500 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 50500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   null sort order: z
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 500 Data size: 100000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 50500 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint), _col2 (type: double)
         Reducer 3 
             Execution mode: llap
@@ -76,14 +76,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 316 Data size: 63200 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 31916 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -92,12 +92,12 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string)
                     outputColumnNames: key, c1, c2
-                    Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       null sort order: 
                       sort order: 
                       Map-reduce partition columns: rand() (type: double)
-                      Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: key (type: string), c1 (type: int), c2 (type: string)
         Reducer 4 
             Execution mode: llap
diff --git a/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out b/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
index 7b6bd1f..54dc91e 100644
--- a/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
@@ -103,26 +103,26 @@ STAGE PLANS:
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
                         null sort order: zz
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(DISTINCT substr(_col1, 5))
                       keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                         null sort order: zzz
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                        Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Map 9 
@@ -140,26 +140,26 @@ STAGE PLANS:
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
                         null sort order: zz
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(DISTINCT substr(_col1, 5))
                       keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                         null sort order: zzz
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                        Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -180,26 +180,26 @@ STAGE PLANS:
                     minReductionHashAggr: 0.0
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string)
                       null sort order: zz
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count(DISTINCT substr(_col1, 5))
                     keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                     minReductionHashAggr: 0.0
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                       null sort order: zzz
                       sort order: +++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                      Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -1082,26 +1082,26 @@ STAGE PLANS:
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
                         null sort order: zz
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(DISTINCT substr(_col1, 5))
                       keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                         null sort order: zzz
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                        Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Map 7 
@@ -1139,26 +1139,26 @@ STAGE PLANS:
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
                         null sort order: zz
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(DISTINCT substr(_col1, 5))
                       keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                         null sort order: zzz
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                        Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 3 
@@ -1281,26 +1281,26 @@ STAGE PLANS:
                     minReductionHashAggr: 0.0
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string)
                       null sort order: zz
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count(DISTINCT substr(_col1, 5))
                     keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                     minReductionHashAggr: 0.0
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                       null sort order: zzz
                       sort order: +++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                      Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
         Union 2 
             Vertex: Union 2
 
@@ -2077,26 +2077,26 @@ STAGE PLANS:
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
                         null sort order: zz
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(DISTINCT substr(_col1, 5))
                       keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                         null sort order: zzz
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                        Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Map 7 
@@ -2134,26 +2134,26 @@ STAGE PLANS:
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
                         null sort order: zz
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(DISTINCT substr(_col1, 5))
                       keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                         null sort order: zzz
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                        Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 3 
@@ -2276,26 +2276,26 @@ STAGE PLANS:
                     minReductionHashAggr: 0.0
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string)
                       null sort order: zz
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 500 Data size: 140000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 137500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count(DISTINCT substr(_col1, 5))
                     keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                     minReductionHashAggr: 0.0
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                       null sort order: zzz
                       sort order: +++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                      Statistics: Num rows: 1001 Data size: 464464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1001 Data size: 459459 Basic stats: COMPLETE Column stats: COMPLETE
         Union 2 
             Vertex: Union 2
 
@@ -3083,26 +3083,26 @@ STAGE PLANS:
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 250 Data size: 70000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 68750 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
                         null sort order: zz
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 250 Data size: 70000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 250 Data size: 68750 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(DISTINCT substr(_col1, 5))
                       keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 501 Data size: 232464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 501 Data size: 229959 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                         null sort order: zzz
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                        Statistics: Num rows: 501 Data size: 232464 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 501 Data size: 229959 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -3123,26 +3123,26 @@ STAGE PLANS:
                     minReductionHashAggr: 0.0
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 250 Data size: 70000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 250 Data size: 68750 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string)
                       null sort order: zz
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 250 Data size: 70000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 68750 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count(DISTINCT substr(_col1, 5))
                     keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                     minReductionHashAggr: 0.0
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 501 Data size: 232464 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 501 Data size: 229959 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                       null sort order: zzz
                       sort order: +++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                      Statistics: Num rows: 501 Data size: 232464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 501 Data size: 229959 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -4072,19 +4072,19 @@ STAGE PLANS:
                   minReductionHashAggr: 0.0
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 125 Data size: 35000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 125 Data size: 34375 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 125 Data size: 35000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 125 Data size: 34375 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(DISTINCT substr(_col1, 5))
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: complete
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 250 Data size: 116000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 114750 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string), CAST( _col2 AS STRING) (type: string)
                     outputColumnNames: _col0, _col1, _col2
diff --git a/ql/src/test/results/clientpositive/llap/udf_coalesce.q.out b/ql/src/test/results/clientpositive/llap/udf_coalesce.q.out
index f35ec7a..a934fdc 100644
--- a/ql/src/test/results/clientpositive/llap/udf_coalesce.q.out
+++ b/ql/src/test/results/clientpositive/llap/udf_coalesce.q.out
@@ -61,20 +61,39 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 #### A masked pattern was here ####
 STAGE DEPENDENCIES:
-  Stage-0 is a root stage
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: 1 (type: int), 1 (type: int), 2 (type: int), 1 (type: int), 3 (type: int), 4 (type: int), '1' (type: string), '1' (type: string), '2' (type: string), '1' (type: string), '3' (type: string), '4' (type: string), 1 (type: decimal(1,0)), 1 (type: decimal(1,0)), 2 (type: decimal(1,0)), 2 (type: decimal(1,0)), 2 (type: decimal(1,0)), null (type: int)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
+                    Statistics: Num rows: 500 Data size: 547004 Basic stats: COMPLETE Column stats: COMPLETE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 500 Data size: 547004 Basic stats: COMPLETE Column stats: COMPLETE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+
   Stage: Stage-0
     Fetch Operator
       limit: -1
       Processor Tree:
-        TableScan
-          alias: src
-          Row Limit Per Split: 1
-          Select Operator
-            expressions: 1 (type: int), 1 (type: int), 2 (type: int), 1 (type: int), 3 (type: int), 4 (type: int), '1' (type: string), '1' (type: string), '2' (type: string), '1' (type: string), '3' (type: string), '4' (type: string), 1 (type: decimal(1,0)), 1 (type: decimal(1,0)), 2 (type: decimal(1,0)), 2 (type: decimal(1,0)), 2 (type: decimal(1,0)), null (type: int)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
-            ListSink
+        ListSink
 
 PREHOOK: query: SELECT COALESCE(1),
        COALESCE(1, 2),
@@ -122,6 +141,53 @@ POSTHOOK: Input: default@src
 #### A masked pattern was here ####
 1	1	2	1	3	4	1	1	2	1	3	4	1	1	2	2	2	NULL
 PREHOOK: query: EXPLAIN
+SELECT COALESCE(key,'x') from src limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN
+SELECT COALESCE(key,'x') from src limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: COALESCE(key,'x') (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 1
+                      Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE
+                        table:
+                            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN
 SELECT COALESCE(src_thrift.lint[1], 999),
        COALESCE(src_thrift.lintstring[0].mystring, '999'),
        COALESCE(src_thrift.mstringstring['key_2'], '999')
@@ -138,19 +204,38 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src_thrift
 #### A masked pattern was here ####
 STAGE DEPENDENCIES:
-  Stage-0 is a root stage
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src_thrift
+                  Statistics: Num rows: 11 Data size: 39600 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: COALESCE(lint[1],999) (type: int), COALESCE(lintstring[0].mystring,'999') (type: string), COALESCE(mstringstring['key_2'],'999') (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 11 Data size: 39600 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 11 Data size: 39600 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+
   Stage: Stage-0
     Fetch Operator
       limit: -1
       Processor Tree:
-        TableScan
-          alias: src_thrift
-          Select Operator
-            expressions: COALESCE(lint[1],999) (type: int), COALESCE(lintstring[0].mystring,'999') (type: string), COALESCE(mstringstring['key_2'],'999') (type: string)
-            outputColumnNames: _col0, _col1, _col2
-            ListSink
+        ListSink
 
 PREHOOK: query: SELECT COALESCE(src_thrift.lint[1], 999),
        COALESCE(src_thrift.lintstring[0].mystring, '999'),
diff --git a/ql/src/test/results/clientpositive/llap/unionDistinct_1.q.out b/ql/src/test/results/clientpositive/llap/unionDistinct_1.q.out
index 6c213f0..aa92f46 100644
--- a/ql/src/test/results/clientpositive/llap/unionDistinct_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/unionDistinct_1.q.out
@@ -1147,19 +1147,19 @@ STAGE PLANS:
                   minReductionHashAggr: 0.0
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 125 Data size: 35500 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 125 Data size: 34875 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 125 Data size: 35500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 125 Data size: 34875 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(DISTINCT substr(_col1, 5))
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: complete
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 250 Data size: 117000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 115750 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string), CAST( _col2 AS STRING) (type: string)
                     outputColumnNames: _col0, _col1, _col2
diff --git a/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out b/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out
index b81c166..7adb53b 100644
--- a/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out
@@ -216,13 +216,13 @@ STAGE PLANS:
                         native: true
                         projectedOutputColumnNums: [4, 21, 26, 30, 34, 38, 42, 44, 46, 48, 50, 52, 54, 58, 61, 64, 67]
                         selectExpressions: VectorUDFAdaptor(CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, LongColLessLongScalar(col 4:int, val 100) -> 20:boolean) -> 21:stri [...]
-                    Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
                       File Sink Vectorization:
                           className: VectorFileSinkOperator
                           native: false
-                      Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
                       table:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -553,13 +553,13 @@ STAGE PLANS:
                         native: true
                         projectedOutputColumnNums: [4, 24, 33, 40, 44, 48, 52, 54, 56, 58, 60, 62, 64, 68, 71, 74, 77]
                         selectExpressions: IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 22:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 21:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprStringScalarStringS [...]
-                    Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
                       File Sink Vectorization:
                           className: VectorFileSinkOperator
                           native: false
-                      Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
                       table:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -890,13 +890,13 @@ STAGE PLANS:
                         native: true
                         projectedOutputColumnNums: [4, 27, 39, 48, 52, 57, 62, 64, 66, 71, 76, 78, 80, 84, 87, 90, 93]
                         selectExpressions: IfExprColumnCondExpr(col 17:boolean, col 18:stringcol 26:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, ConstantVectorExpression(val Single) -> 18:string, IfExprColumnCondExpr(col 19:boolean, col 20:stringcol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 19:boolean, ConstantVectorExpression(val Two) -> 20:string, IfExprColumnCondExpr(col 21:boolean, col 22:stringcol 24:string)(children: LongColLessLo [...]
-                    Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
                       File Sink Vectorization:
                           className: VectorFileSinkOperator
                           native: false
-                      Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
                       table:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/vector_case_when_2.q.out b/ql/src/test/results/clientpositive/llap/vector_case_when_2.q.out
index bcc7dd2..cc72f45 100644
--- a/ql/src/test/results/clientpositive/llap/vector_case_when_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_case_when_2.q.out
@@ -151,7 +151,7 @@ STAGE PLANS:
                         native: true
                         projectedOutputColumnNums: [1, 3, 9, 14, 18, 22, 24, 27, 32, 38, 2]
                         selectExpressions: VectorUDFAdaptor(CASE WHEN ((ctimestamp2 <= TIMESTAMP'1800-12-31 00:00:00')) THEN ('1800s or Earlier') WHEN ((ctimestamp2 < TIMESTAMP'1900-01-01 00:00:00')) THEN ('1900s') WHEN (ctimestamp2 BETWEEN TIMESTAMP'2006-01-01 00:00:00' AND TIMESTAMP'2010-12-31 23:59:59.999999999') THEN ('Late 2000s') WHEN ((ctimestamp2 <= TIMESTAMP'2015-12-31 23:59:59.999999999')) THEN ('Early 2010s') ELSE ('Unknown') END)(children: TimestampColLessEqualTimestampScalar [...]
-                    Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: timestamp), _col10 (type: string), _col1 (type: timestamp)
                       null sort order: zzz
@@ -162,7 +162,7 @@ STAGE PLANS:
                           native: true
                           nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                           valueColumns: 9:string, 14:string, 18:string, 22:int, 24:string, 27:int, 32:int, 38:date
-                      Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int), _col8 (type: int), _col9 (type: date)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -204,13 +204,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 2, 3, 4, 5, 6, 7, 8, 9, 10]
-                Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -441,7 +441,7 @@ STAGE PLANS:
                         native: true
                         projectedOutputColumnNums: [1, 3, 12, 21, 28, 32, 34, 37, 42, 48, 2]
                         selectExpressions: IfExprStringScalarStringGroupColumn(col 5:boolean, val 1800s or Earliercol 11:string)(children: TimestampColLessEqualTimestampScalar(col 3:timestamp, val 1800-12-31 00:00:00) -> 5:boolean, IfExprStringScalarStringGroupColumn(col 6:boolean, val 1900scol 10:string)(children: TimestampColLessTimestampScalar(col 3:timestamp, val 1900-01-01 00:00:00) -> 6:boolean, IfExprStringScalarStringGroupColumn(col 7:boolean, val Late 2000scol 9:string)(children [...]
-                    Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: timestamp), _col10 (type: string), _col1 (type: timestamp)
                       null sort order: zzz
@@ -452,7 +452,7 @@ STAGE PLANS:
                           native: true
                           nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                           valueColumns: 12:string, 21:string, 28:string, 32:int, 34:string, 37:int, 42:int, 48:date
-                      Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int), _col8 (type: int), _col9 (type: date)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -494,13 +494,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 2, 3, 4, 5, 6, 7, 8, 9, 10]
-                Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -731,7 +731,7 @@ STAGE PLANS:
                         native: true
                         projectedOutputColumnNums: [1, 3, 15, 27, 36, 40, 42, 45, 50, 56, 2]
                         selectExpressions: IfExprColumnCondExpr(col 5:boolean, col 6:stringcol 14:string)(children: TimestampColLessEqualTimestampScalar(col 3:timestamp, val 1800-12-31 00:00:00) -> 5:boolean, ConstantVectorExpression(val 1800s or Earlier) -> 6:string, IfExprColumnCondExpr(col 7:boolean, col 8:stringcol 13:string)(children: TimestampColLessTimestampScalar(col 3:timestamp, val 1900-01-01 00:00:00) -> 7:boolean, ConstantVectorExpression(val 1900s) -> 8:string, IfExprColumnC [...]
-                    Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: timestamp), _col10 (type: string), _col1 (type: timestamp)
                       null sort order: zzz
@@ -742,7 +742,7 @@ STAGE PLANS:
                           native: true
                           nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                           valueColumns: 15:string, 27:string, 36:string, 40:int, 42:string, 45:int, 50:int, 56:date
-                      Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int), _col8 (type: int), _col9 (type: date)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -784,13 +784,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 2, 3, 4, 5, 6, 7, 8, 9, 10]
-                Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/vector_groupby4.q.out b/ql/src/test/results/clientpositive/llap/vector_groupby4.q.out
index e31fe56..f8d6e2d 100644
--- a/ql/src/test/results/clientpositive/llap/vector_groupby4.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_groupby4.q.out
@@ -108,7 +108,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: partial1
                 outputColumnNames: _col0
-                Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   null sort order: z
@@ -118,7 +118,7 @@ STAGE PLANS:
                       className: VectorReduceSinkStringOperator
                       native: true
                       nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Vectorization:
@@ -139,13 +139,13 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: final
                 outputColumnNames: _col0
-                Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                       output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
@@ -158,7 +158,7 @@ STAGE PLANS:
                       className: VectorSelectOperator
                       native: true
                       projectedOutputColumnNums: [0]
-                  Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     null sort order: 
                     sort order: 
@@ -167,7 +167,7 @@ STAGE PLANS:
                         className: VectorReduceSinkObjectHashOperator
                         native: true
                         nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                    Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: c1 (type: string)
         Reducer 4 
             Execution mode: llap
diff --git a/ql/src/test/results/clientpositive/llap/vector_groupby6.q.out b/ql/src/test/results/clientpositive/llap/vector_groupby6.q.out
index 25b6be6..2042f6d 100644
--- a/ql/src/test/results/clientpositive/llap/vector_groupby6.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_groupby6.q.out
@@ -108,7 +108,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: partial1
                 outputColumnNames: _col0
-                Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   null sort order: z
@@ -118,7 +118,7 @@ STAGE PLANS:
                       className: VectorReduceSinkStringOperator
                       native: true
                       nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Vectorization:
@@ -139,13 +139,13 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: final
                 outputColumnNames: _col0
-                Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                       output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
@@ -158,7 +158,7 @@ STAGE PLANS:
                       className: VectorSelectOperator
                       native: true
                       projectedOutputColumnNums: [0]
-                  Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     null sort order: 
                     sort order: 
@@ -167,7 +167,7 @@ STAGE PLANS:
                         className: VectorReduceSinkObjectHashOperator
                         native: true
                         nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                    Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: c1 (type: string)
         Reducer 4 
             Execution mode: llap
diff --git a/ql/src/test/results/clientpositive/llap/vector_if_expr.q.out b/ql/src/test/results/clientpositive/llap/vector_if_expr.q.out
index 5d955bc..ed73482 100644
--- a/ql/src/test/results/clientpositive/llap/vector_if_expr.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_if_expr.q.out
@@ -47,7 +47,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [10, 13]
                           selectExpressions: IfExprStringScalarStringScalar(col 10:boolean, val first, val second) -> 13:string
-                      Statistics: Num rows: 4587 Data size: 857712 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4587 Data size: 426534 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: boolean)
                         null sort order: z
@@ -56,7 +56,7 @@ STAGE PLANS:
                             className: VectorReduceSinkObjectHashOperator
                             native: true
                             nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                        Statistics: Num rows: 4587 Data size: 857712 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4587 Data size: 426534 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -85,13 +85,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1]
-                Statistics: Num rows: 4587 Data size: 857712 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4587 Data size: 426534 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 4587 Data size: 857712 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4587 Data size: 426534 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/vector_nvl.q.out b/ql/src/test/results/clientpositive/llap/vector_nvl.q.out
index 420520e..5346ed4 100644
--- a/ql/src/test/results/clientpositive/llap/vector_nvl.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_nvl.q.out
@@ -147,19 +147,19 @@ STAGE PLANS:
                         native: true
                         projectedOutputColumnNums: [4, 14]
                         selectExpressions: VectorCoalesce(columns [4, 13])(children: col 4:float, ConstantVectorExpression(val 1.0) -> 13:float) -> 14:float
-                    Statistics: Num rows: 12288 Data size: 85848 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12288 Data size: 73392 Basic stats: COMPLETE Column stats: COMPLETE
                     Limit
                       Number of rows: 10
                       Limit Vectorization:
                           className: VectorLimitOperator
                           native: true
-                      Statistics: Num rows: 10 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 10 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
                         File Sink Vectorization:
                             className: VectorFileSinkOperator
                             native: false
-                        Statistics: Num rows: 10 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 10 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/vector_udf1.q.out b/ql/src/test/results/clientpositive/llap/vector_udf1.q.out
index 956ff47..8ca8a8e 100644
--- a/ql/src/test/results/clientpositive/llap/vector_udf1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_udf1.q.out
@@ -2585,19 +2585,19 @@ STAGE PLANS:
                         native: true
                         projectedOutputColumnNums: [9, 10, 13]
                         selectExpressions: StringSubstrColStartLen(col 1:string, start 0, length 3) -> 9:string, StringSubstrColStartLen(col 3:varchar(20), start 0, length 3) -> 10:string, StringGroupColEqualStringGroupColumn(col 11:string, col 12:string)(children: StringSubstrColStartLen(col 1:string, start 0, length 3) -> 11:string, StringSubstrColStartLen(col 3:varchar(20), start 0, length 3) -> 12:string) -> 13:boolean
-                    Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                     Limit
                       Number of rows: 1
                       Limit Vectorization:
                           className: VectorLimitOperator
                           native: true
-                      Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
                         File Sink Vectorization:
                             className: VectorFileSinkOperator
                             native: false
-                        Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_string_funcs.q.out b/ql/src/test/results/clientpositive/llap/vectorized_string_funcs.q.out
index 29e35ec..12e5037 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_string_funcs.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_string_funcs.q.out
@@ -71,10 +71,10 @@ STAGE PLANS:
                     Select Operator
                       expressions: substr(cstring1, 1, 2) (type: string), substr(cstring1, 2) (type: string), lower(cstring1) (type: string), upper(cstring1) (type: string), upper(cstring1) (type: string), length(cstring1) (type: int), trim(cstring1) (type: string), ltrim(cstring1) (type: string), rtrim(cstring1) (type: string), concat(cstring1, cstring2) (type: string), concat('>', cstring1) (type: string), concat(cstring1, '<') (type: string), concat(substr(cstring1, 1, 2), substr(cstr [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                      Statistics: Num rows: 1024 Data size: 2265088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1024 Data size: 2024426 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 1024 Data size: 2265088 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1024 Data size: 2024426 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
index 0077f08..0e0058b 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
@@ -274,7 +274,7 @@ STAGE PLANS:
                         native: true
                         projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 0, 1, 3, 13, 14, 15, 16, 17]
                         selectExpressions: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 6:int, VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 7:int, VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 8:int, VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 9:int, VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 10:int, VectorUDFMinuteTimestamp(col 1:timestam [...]
-                    Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 52 Data size: 11916 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
                       null sort order: z
@@ -283,7 +283,7 @@ STAGE PLANS:
                           className: VectorReduceSinkObjectHashOperator
                           native: true
                           nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 52 Data size: 11916 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int), _col9 (type: boolean), _col10 (type: timestamp), _col11 (type: timestamp), _col12 (type: timestamp), _col13 (type: timestamp), _col14 (type: timestamp), _col15 (type: timestamp), _col16 (type: timestamp)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -312,13 +312,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15]
-                Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 52 Data size: 11916 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 11916 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/merge_dynamic_partition4.q.out b/ql/src/test/results/clientpositive/merge_dynamic_partition4.q.out
index c55ef0f..8387880 100644
--- a/ql/src/test/results/clientpositive/merge_dynamic_partition4.q.out
+++ b/ql/src/test/results/clientpositive/merge_dynamic_partition4.q.out
@@ -140,14 +140,13 @@ POSTHOOK: Input: default@srcpart_merge_dp_rc_n1@ds=2008-04-08/hr=11
 POSTHOOK: Input: default@srcpart_merge_dp_rc_n1@ds=2008-04-08/hr=12
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-8 depends on stages: Stage-2 , consists of Stage-5, Stage-4, Stage-6
-  Stage-5
-  Stage-0 depends on stages: Stage-5, Stage-4, Stage-7
-  Stage-3 depends on stages: Stage-0
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-6
-  Stage-7 depends on stages: Stage-6
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+  Stage-2 depends on stages: Stage-0
+  Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
   Stage: Stage-1
@@ -160,79 +159,56 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), value (type: string), if(((UDFToDouble(key) % 2.0D) = 0.0D), 'a1', 'b1') (type: string)
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 1000 Data size: 264000 Basic stats: COMPLETE Column stats: COMPLETE
+                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.merge_dynamic_part_n3
               Select Operator
                 expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col2 (type: string)
                 outputColumnNames: key, value, ds, hr
-                Statistics: Num rows: 1000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 358000 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                   keys: ds (type: string), hr (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
-              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
       Reduce Operator Tree:
         Group By Operator
           aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1)
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: string), _col1 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col2 (type: string)
-              null sort order: a
-              sort order: +
-              Map-reduce partition columns: _col2 (type: string)
-              Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col0 (type: string), _col1 (type: string)
-      Reduce Operator Tree:
-        Select Operator
-          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), KEY._col2 (type: string)
-          outputColumnNames: _col0, _col1, _col2
-          File Output Operator
-            compressed: false
-            Dp Sort State: PARTITION_SORTED
-            Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
-            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.merge_dynamic_part_n3
-
-  Stage: Stage-8
+  Stage: Stage-7
     Conditional Operator
 
-  Stage: Stage-5
+  Stage: Stage-4
     Move Operator
       files:
           hdfs directory: true
@@ -251,7 +227,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
               name: default.merge_dynamic_part_n3
 
-  Stage: Stage-3
+  Stage: Stage-2
     Stats Work
       Basic Stats Work:
       Column Stats Desc:
@@ -259,21 +235,21 @@ STAGE PLANS:
           Column Types: string, string
           Table: default.merge_dynamic_part_n3
 
-  Stage: Stage-4
+  Stage: Stage-3
     Merge File Operator
       Map Operator Tree:
           RCFile Merge Operator
       merge level: block
       input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
 
-  Stage: Stage-6
+  Stage: Stage-5
     Merge File Operator
       Map Operator Tree:
           RCFile Merge Operator
       merge level: block
       input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
 
-  Stage: Stage-7
+  Stage: Stage-6
     Move Operator
       files:
           hdfs directory: true
diff --git a/ql/src/test/results/clientpositive/merge_dynamic_partition5.q.out b/ql/src/test/results/clientpositive/merge_dynamic_partition5.q.out
index 251c3f8..518f400 100644
--- a/ql/src/test/results/clientpositive/merge_dynamic_partition5.q.out
+++ b/ql/src/test/results/clientpositive/merge_dynamic_partition5.q.out
@@ -116,14 +116,13 @@ POSTHOOK: Input: default@srcpart_merge_dp_rc@ds=2008-04-08/hr=11
 POSTHOOK: Input: default@srcpart_merge_dp_rc@ds=2008-04-08/hr=12
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-8 depends on stages: Stage-2 , consists of Stage-5, Stage-4, Stage-6
-  Stage-5
-  Stage-0 depends on stages: Stage-5, Stage-4, Stage-7
-  Stage-3 depends on stages: Stage-0
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-6
-  Stage-7 depends on stages: Stage-6
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+  Stage-2 depends on stages: Stage-0
+  Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
   Stage: Stage-1
@@ -136,79 +135,56 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), value (type: string), if(((UDFToDouble(key) % 100.0D) = 0.0D), 'a1', 'b1') (type: string)
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 618 Data size: 223716 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 618 Data size: 163152 Basic stats: COMPLETE Column stats: COMPLETE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 618 Data size: 163152 Basic stats: COMPLETE Column stats: COMPLETE
+                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.merge_dynamic_part
               Select Operator
                 expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col2 (type: string)
                 outputColumnNames: key, value, ds, hr
-                Statistics: Num rows: 618 Data size: 281808 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 618 Data size: 221244 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                   keys: ds (type: string), hr (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 309 Data size: 357822 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 309 Data size: 357822 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
-              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
       Reduce Operator Tree:
         Group By Operator
           aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1)
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 309 Data size: 357822 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: string), _col1 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 309 Data size: 357822 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 309 Data size: 357822 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 1060 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col2 (type: string)
-              null sort order: a
-              sort order: +
-              Map-reduce partition columns: _col2 (type: string)
-              Statistics: Num rows: 618 Data size: 223716 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col0 (type: string), _col1 (type: string)
-      Reduce Operator Tree:
-        Select Operator
-          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), KEY._col2 (type: string)
-          outputColumnNames: _col0, _col1, _col2
-          File Output Operator
-            compressed: false
-            Dp Sort State: PARTITION_SORTED
-            Statistics: Num rows: 618 Data size: 223716 Basic stats: COMPLETE Column stats: COMPLETE
-            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.merge_dynamic_part
-
-  Stage: Stage-8
+  Stage: Stage-7
     Conditional Operator
 
-  Stage: Stage-5
+  Stage: Stage-4
     Move Operator
       files:
           hdfs directory: true
@@ -227,7 +203,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
               name: default.merge_dynamic_part
 
-  Stage: Stage-3
+  Stage: Stage-2
     Stats Work
       Basic Stats Work:
       Column Stats Desc:
@@ -235,21 +211,21 @@ STAGE PLANS:
           Column Types: string, string
           Table: default.merge_dynamic_part
 
-  Stage: Stage-4
+  Stage: Stage-3
     Merge File Operator
       Map Operator Tree:
           RCFile Merge Operator
       merge level: block
       input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
 
-  Stage: Stage-6
+  Stage: Stage-5
     Merge File Operator
       Map Operator Tree:
           RCFile Merge Operator
       merge level: block
       input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
 
-  Stage: Stage-7
+  Stage: Stage-6
     Move Operator
       files:
           hdfs directory: true
diff --git a/ql/src/test/results/clientpositive/nullgroup4_multi_distinct.q.out b/ql/src/test/results/clientpositive/nullgroup4_multi_distinct.q.out
index 318e694..b4ff434 100644
--- a/ql/src/test/results/clientpositive/nullgroup4_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/nullgroup4_multi_distinct.q.out
@@ -33,12 +33,12 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                  Statistics: Num rows: 125 Data size: 37375 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 125 Data size: 25125 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
-                    Statistics: Num rows: 125 Data size: 37375 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 125 Data size: 25125 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
diff --git a/ql/src/test/results/clientpositive/offset_limit_global_optimizer.q.out b/ql/src/test/results/clientpositive/offset_limit_global_optimizer.q.out
index e29ca9d..7e1a2c0 100644
--- a/ql/src/test/results/clientpositive/offset_limit_global_optimizer.q.out
+++ b/ql/src/test/results/clientpositive/offset_limit_global_optimizer.q.out
@@ -36,12 +36,12 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), substr(value, 5) (type: string), ds (type: string), hr (type: string)
               outputColumnNames: _col0, _col1, _col2, _col3
-              Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                 null sort order: zzzz
                 sort order: ++++
-                Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
                 tag: -1
                 TopN: 410
                 TopN Hash Memory Usage: 0.1
@@ -256,17 +256,17 @@ STAGE PLANS:
         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: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 10
             Offset of rows: 400
-            Statistics: Num rows: 10 Data size: 6390 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 5410 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 10 Data size: 6390 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 10 Data size: 5410 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -354,12 +354,12 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), substr(value, 5) (type: string), ds (type: string), hr (type: string)
               outputColumnNames: _col0, _col1, _col2, _col3
-              Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                 null sort order: zzzz
                 sort order: ++++
-                Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
                 tag: -1
                 TopN: 500
                 TopN Hash Memory Usage: 0.1
@@ -574,17 +574,17 @@ STAGE PLANS:
         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: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 10
             Offset of rows: 490
-            Statistics: Num rows: 10 Data size: 6390 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 5410 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 10 Data size: 6390 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 10 Data size: 5410 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -672,12 +672,12 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), substr(value, 5) (type: string), ds (type: string), hr (type: string)
               outputColumnNames: _col0, _col1, _col2, _col3
-              Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                 null sort order: zzzz
                 sort order: ++++
-                Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
                 tag: -1
                 TopN: 510
                 TopN Hash Memory Usage: 0.1
@@ -892,17 +892,17 @@ STAGE PLANS:
         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: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 20
             Offset of rows: 490
-            Statistics: Num rows: 20 Data size: 12780 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 10820 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 20 Data size: 12780 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 10820 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -1000,12 +1000,12 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), substr(value, 5) (type: string), ds (type: string), hr (type: string)
               outputColumnNames: _col0, _col1, _col2, _col3
-              Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                 null sort order: zzzz
                 sort order: ++++
-                Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
                 tag: -1
                 TopN: 1090
                 TopN Hash Memory Usage: 0.1
@@ -1220,17 +1220,17 @@ STAGE PLANS:
         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: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 600
             Offset of rows: 490
-            Statistics: Num rows: 600 Data size: 383400 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 600 Data size: 324600 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 600 Data size: 383400 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 600 Data size: 324600 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -1903,12 +1903,12 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), substr(value, 5) (type: string), ds (type: string), hr (type: string)
               outputColumnNames: _col0, _col1, _col2, _col3
-              Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                 null sort order: zzzz
                 sort order: ++++
-                Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
                 tag: -1
                 TopN: 410
                 TopN Hash Memory Usage: 0.1
@@ -2123,17 +2123,17 @@ STAGE PLANS:
         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: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 10
             Offset of rows: 400
-            Statistics: Num rows: 10 Data size: 6390 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 5410 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 10 Data size: 6390 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 10 Data size: 5410 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -2216,12 +2216,12 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), substr(value, 5) (type: string), ds (type: string), hr (type: string)
               outputColumnNames: _col0, _col1, _col2, _col3
-              Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                 null sort order: zzzz
                 sort order: ++++
-                Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
                 tag: -1
                 TopN: 500
                 TopN Hash Memory Usage: 0.1
@@ -2436,17 +2436,17 @@ STAGE PLANS:
         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: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 10
             Offset of rows: 490
-            Statistics: Num rows: 10 Data size: 6390 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 5410 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 10 Data size: 6390 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 10 Data size: 5410 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -2529,12 +2529,12 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), substr(value, 5) (type: string), ds (type: string), hr (type: string)
               outputColumnNames: _col0, _col1, _col2, _col3
-              Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                 null sort order: zzzz
                 sort order: ++++
-                Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
                 tag: -1
                 TopN: 510
                 TopN Hash Memory Usage: 0.1
@@ -2749,17 +2749,17 @@ STAGE PLANS:
         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: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 20
             Offset of rows: 490
-            Statistics: Num rows: 20 Data size: 12780 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 10820 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 20 Data size: 12780 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 10820 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -2852,12 +2852,12 @@ STAGE PLANS:
             Select Operator
               expressions: key (type: string), substr(value, 5) (type: string), ds (type: string), hr (type: string)
               outputColumnNames: _col0, _col1, _col2, _col3
-              Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                 null sort order: zzzz
                 sort order: ++++
-                Statistics: Num rows: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
                 tag: -1
                 TopN: 1090
                 TopN Hash Memory Usage: 0.1
@@ -3072,17 +3072,17 @@ STAGE PLANS:
         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: 2000 Data size: 1278000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2000 Data size: 1082000 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 600
             Offset of rows: 490
-            Statistics: Num rows: 600 Data size: 383400 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 600 Data size: 324600 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
-              Statistics: Num rows: 600 Data size: 383400 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 600 Data size: 324600 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query19.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query19.q.out
index 94bf30c..4000b81 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query19.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query19.q.out
@@ -96,23 +96,23 @@ Stage-0
                       PartitionCols:_col0, _col1, _col2, _col3
                       Group By Operator [GBY_34] (rows=76645658 width=314)
                         Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col8)"],keys:_col12, _col11, _col13, _col14
-                        Select Operator [SEL_33] (rows=76645658 width=650)
+                        Select Operator [SEL_33] (rows=76645658 width=458)
                           Output:["_col8","_col11","_col12","_col13","_col14"]
-                          Filter Operator [FIL_32] (rows=76645658 width=650)
+                          Filter Operator [FIL_32] (rows=76645658 width=458)
                             predicate:(_col3 <> _col16)
-                            Merge Join Operator [MERGEJOIN_122] (rows=76645658 width=650)
+                            Merge Join Operator [MERGEJOIN_122] (rows=76645658 width=458)
                               Conds:RS_29._col7=RS_143._col0(Inner),Output:["_col3","_col8","_col11","_col12","_col13","_col14","_col16"]
                             <-Map 14 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_143]
                                 PartitionCols:_col0
-                                Select Operator [SEL_142] (rows=1704 width=188)
+                                Select Operator [SEL_142] (rows=1704 width=92)
                                   Output:["_col0","_col1"]
                                   TableScan [TS_21] (rows=1704 width=93)
                                     default@store,store,Tbl:COMPLETE,Col:COMPLETE,Output:["s_store_sk","s_zip"]
                             <-Reducer 3 [SIMPLE_EDGE]
                               SHUFFLE [RS_29]
                                 PartitionCols:_col7
-                                Merge Join Operator [MERGEJOIN_121] (rows=76645658 width=468)
+                                Merge Join Operator [MERGEJOIN_121] (rows=76645658 width=372)
                                   Conds:RS_26._col0=RS_27._col2(Inner),Output:["_col3","_col7","_col8","_col11","_col12","_col13","_col14"]
                                 <-Reducer 10 [SIMPLE_EDGE]
                                   SHUFFLE [RS_27]
@@ -165,7 +165,7 @@ Stage-0
                                 <-Reducer 2 [SIMPLE_EDGE]
                                   SHUFFLE [RS_26]
                                     PartitionCols:_col0
-                                    Merge Join Operator [MERGEJOIN_118] (rows=80000000 width=188)
+                                    Merge Join Operator [MERGEJOIN_118] (rows=80000000 width=92)
                                       Conds:RS_125._col1=RS_127._col0(Inner),Output:["_col0","_col3"]
                                     <-Map 1 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_125]
@@ -179,7 +179,7 @@ Stage-0
                                     <-Map 7 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_127]
                                         PartitionCols:_col0
-                                        Select Operator [SEL_126] (rows=40000000 width=188)
+                                        Select Operator [SEL_126] (rows=40000000 width=92)
                                           Output:["_col0","_col1"]
                                           TableScan [TS_3] (rows=40000000 width=93)
                                             default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_zip"]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query79.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query79.q.out
index 2828fb0..2b94615 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query79.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query79.q.out
@@ -72,17 +72,17 @@ Stage-0
     Stage-1
       Reducer 3 vectorized
       File Output Operator [FS_125]
-        Limit [LIM_124] (rows=100 width=776)
+        Limit [LIM_124] (rows=100 width=592)
           Number of rows:100
-          Select Operator [SEL_123] (rows=479121995 width=776)
+          Select Operator [SEL_123] (rows=479121995 width=592)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           <-Reducer 2 [SIMPLE_EDGE]
             SHUFFLE [RS_32]
-              Select Operator [SEL_31] (rows=479121995 width=776)
+              Select Operator [SEL_31] (rows=479121995 width=592)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                Top N Key Operator [TNK_56] (rows=479121995 width=685)
+                Top N Key Operator [TNK_56] (rows=479121995 width=593)
                   keys:_col2, _col1, substr(_col5, 1, 30), _col7,top n:100
-                  Merge Join Operator [MERGEJOIN_100] (rows=479121995 width=685)
+                  Merge Join Operator [MERGEJOIN_100] (rows=479121995 width=593)
                     Conds:RS_102._col0=RS_122._col1(Inner),Output:["_col1","_col2","_col3","_col5","_col6","_col7","_col8"]
                   <-Map 1 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_102]
@@ -94,7 +94,7 @@ Stage-0
                   <-Reducer 8 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_122]
                       PartitionCols:_col1
-                      Select Operator [SEL_121] (rows=479121995 width=508)
+                      Select Operator [SEL_121] (rows=479121995 width=416)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                         Group By Operator [GBY_120] (rows=479121995 width=328)
                           Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query8.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query8.q.out
index ed1501b..06b9f6f 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query8.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query8.q.out
@@ -271,7 +271,7 @@ Stage-0
                           <-Map 18 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_149]
                               PartitionCols:_col2
-                              Select Operator [SEL_148] (rows=1704 width=276)
+                              Select Operator [SEL_148] (rows=1704 width=178)
                                 Output:["_col0","_col1","_col2"]
                                 Filter Operator [FIL_147] (rows=1704 width=181)
                                   predicate:substr(s_zip, 1, 2) is not null
@@ -280,24 +280,24 @@ Stage-0
                           <-Reducer 11 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_146]
                               PartitionCols:_col0
-                              Select Operator [SEL_145] (rows=1 width=184)
+                              Select Operator [SEL_145] (rows=1 width=86)
                                 Output:["_col0"]
-                                Filter Operator [FIL_144] (rows=1 width=192)
+                                Filter Operator [FIL_144] (rows=1 width=96)
                                   predicate:(_col1 = 2L)
-                                  Group By Operator [GBY_143] (rows=5633 width=192)
+                                  Group By Operator [GBY_143] (rows=5633 width=96)
                                     Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
                                   <-Union 10 [SIMPLE_EDGE]
                                     <-Reducer 16 [CONTAINS] vectorized
                                       Reduce Output Operator [RS_175]
                                         PartitionCols:_col0
-                                        Group By Operator [GBY_174] (rows=5633 width=192)
+                                        Group By Operator [GBY_174] (rows=5633 width=96)
                                           Output:["_col0","_col1"],aggregations:["count(_col1)"],keys:_col0
-                                          Group By Operator [GBY_173] (rows=1126 width=192)
+                                          Group By Operator [GBY_173] (rows=1126 width=96)
                                             Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
                                           <-Reducer 15 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_172]
                                               PartitionCols:_col0
-                                              Group By Operator [GBY_171] (rows=1126 width=192)
+                                              Group By Operator [GBY_171] (rows=1126 width=96)
                                                 Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
                                                 Select Operator [SEL_170] (rows=2253 width=97)
                                                   Output:["_col0"]
@@ -333,14 +333,14 @@ Stage-0
                                     <-Reducer 9 [CONTAINS] vectorized
                                       Reduce Output Operator [RS_161]
                                         PartitionCols:_col0
-                                        Group By Operator [GBY_160] (rows=5633 width=192)
+                                        Group By Operator [GBY_160] (rows=5633 width=96)
                                           Output:["_col0","_col1"],aggregations:["count(_col1)"],keys:_col0
-                                          Group By Operator [GBY_159] (rows=10141 width=192)
+                                          Group By Operator [GBY_159] (rows=10141 width=96)
                                             Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
                                           <-Map 8 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_158]
                                               PartitionCols:_col0
-                                              Group By Operator [GBY_157] (rows=141974 width=192)
+                                              Group By Operator [GBY_157] (rows=141974 width=96)
                                                 Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
                                                 Select Operator [SEL_156] (rows=20000000 width=89)
                                                   Output:["_col0"]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query85.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query85.q.out
index 706bbd7..9a550b7 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query85.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query85.q.out
@@ -200,13 +200,13 @@ Stage-0
     Stage-1
       Reducer 6 vectorized
       File Output Operator [FS_209]
-        Limit [LIM_208] (rows=72 width=832)
+        Limit [LIM_208] (rows=72 width=656)
           Number of rows:100
-          Select Operator [SEL_207] (rows=72 width=832)
+          Select Operator [SEL_207] (rows=72 width=656)
             Output:["_col0","_col1","_col2","_col3"]
           <-Reducer 5 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_206]
-              Select Operator [SEL_205] (rows=72 width=832)
+              Select Operator [SEL_205] (rows=72 width=656)
                 Output:["_col4","_col5","_col6","_col7"]
                 Top N Key Operator [TNK_204] (rows=72 width=353)
                   keys:substr(_col0, 1, 20), (UDFToDouble(_col1) / _col2), (_col3 / _col4), (_col5 / _col6),top n:100
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query99.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query99.q.out
index 5310297..98249ac 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query99.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query99.q.out
@@ -96,24 +96,24 @@ Stage-0
     Stage-1
       Reducer 7 vectorized
       File Output Operator [FS_125]
-        Limit [LIM_124] (rows=100 width=590)
+        Limit [LIM_124] (rows=100 width=420)
           Number of rows:100
-          Select Operator [SEL_123] (rows=3920468 width=590)
+          Select Operator [SEL_123] (rows=3920468 width=420)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
           <-Reducer 6 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_122]
-              Select Operator [SEL_121] (rows=3920468 width=590)
+              Select Operator [SEL_121] (rows=3920468 width=420)
                 Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
-                Group By Operator [GBY_120] (rows=3920468 width=406)
+                Group By Operator [GBY_120] (rows=3920468 width=321)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)"],keys:KEY._col0, KEY._col1, KEY._col2
                 <-Reducer 5 [SIMPLE_EDGE]
                   SHUFFLE [RS_26]
                     PartitionCols:_col0, _col1, _col2
-                    Group By Operator [GBY_25] (rows=7840936 width=406)
+                    Group By Operator [GBY_25] (rows=7840936 width=321)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col4)","sum(_col5)","sum(_col6)","sum(_col7)","sum(_col8)"],keys:_col13, _col11, _col15
-                      Top N Key Operator [TNK_56] (rows=15681873 width=386)
+                      Top N Key Operator [TNK_56] (rows=15681873 width=301)
                         keys:_col13, _col11, _col15,top n:100
-                        Merge Join Operator [MERGEJOIN_102] (rows=15681873 width=386)
+                        Merge Join Operator [MERGEJOIN_102] (rows=15681873 width=301)
                           Conds:RS_21._col1=RS_119._col0(Inner),Output:["_col4","_col5","_col6","_col7","_col8","_col11","_col13","_col15"]
                         <-Map 12 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_119]
@@ -125,12 +125,12 @@ Stage-0
                         <-Reducer 4 [SIMPLE_EDGE]
                           SHUFFLE [RS_21]
                             PartitionCols:_col1
-                            Merge Join Operator [MERGEJOIN_101] (rows=15681873 width=291)
+                            Merge Join Operator [MERGEJOIN_101] (rows=15681873 width=206)
                               Conds:RS_18._col3=RS_117._col0(Inner),Output:["_col1","_col4","_col5","_col6","_col7","_col8","_col11","_col13"]
                             <-Map 11 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_117]
                                 PartitionCols:_col0
-                                Select Operator [SEL_116] (rows=27 width=188)
+                                Select Operator [SEL_116] (rows=27 width=103)
                                   Output:["_col0","_col1"]
                                   TableScan [TS_8] (rows=27 width=104)
                                     default@warehouse,warehouse,Tbl:COMPLETE,Col:COMPLETE,Output:["w_warehouse_sk","w_warehouse_name"]
diff --git a/ql/src/test/results/clientpositive/perf/tez/query19.q.out b/ql/src/test/results/clientpositive/perf/tez/query19.q.out
index 55ce944..d94c899 100644
--- a/ql/src/test/results/clientpositive/perf/tez/query19.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/query19.q.out
@@ -96,16 +96,16 @@ Stage-0
                       PartitionCols:_col0, _col1, _col2, _col3
                       Group By Operator [GBY_36] (rows=76645658 width=314)
                         Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col8)"],keys:_col12, _col11, _col13, _col14
-                        Select Operator [SEL_35] (rows=76645658 width=650)
+                        Select Operator [SEL_35] (rows=76645658 width=458)
                           Output:["_col8","_col11","_col12","_col13","_col14"]
-                          Filter Operator [FIL_34] (rows=76645658 width=650)
+                          Filter Operator [FIL_34] (rows=76645658 width=458)
                             predicate:(_col3 <> _col16)
-                            Merge Join Operator [MERGEJOIN_124] (rows=76645658 width=650)
+                            Merge Join Operator [MERGEJOIN_124] (rows=76645658 width=458)
                               Conds:RS_31._col7=RS_147._col0(Inner),Output:["_col3","_col8","_col11","_col12","_col13","_col14","_col16"]
                             <-Map 14 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_147]
                                 PartitionCols:_col0
-                                Select Operator [SEL_146] (rows=1704 width=188)
+                                Select Operator [SEL_146] (rows=1704 width=92)
                                   Output:["_col0","_col1"]
                                   Filter Operator [FIL_145] (rows=1704 width=93)
                                     predicate:s_store_sk is not null
@@ -114,7 +114,7 @@ Stage-0
                             <-Reducer 3 [SIMPLE_EDGE]
                               SHUFFLE [RS_31]
                                 PartitionCols:_col7
-                                Merge Join Operator [MERGEJOIN_123] (rows=76645658 width=468)
+                                Merge Join Operator [MERGEJOIN_123] (rows=76645658 width=372)
                                   Conds:RS_28._col0=RS_29._col2(Inner),Output:["_col3","_col7","_col8","_col11","_col12","_col13","_col14"]
                                 <-Reducer 10 [SIMPLE_EDGE]
                                   SHUFFLE [RS_29]
@@ -167,7 +167,7 @@ Stage-0
                                 <-Reducer 2 [SIMPLE_EDGE]
                                   SHUFFLE [RS_28]
                                     PartitionCols:_col0
-                                    Merge Join Operator [MERGEJOIN_120] (rows=80000000 width=188)
+                                    Merge Join Operator [MERGEJOIN_120] (rows=80000000 width=92)
                                       Conds:RS_127._col1=RS_130._col0(Inner),Output:["_col0","_col3"]
                                     <-Map 1 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_127]
@@ -181,7 +181,7 @@ Stage-0
                                     <-Map 7 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_130]
                                         PartitionCols:_col0
-                                        Select Operator [SEL_129] (rows=40000000 width=188)
+                                        Select Operator [SEL_129] (rows=40000000 width=92)
                                           Output:["_col0","_col1"]
                                           Filter Operator [FIL_128] (rows=40000000 width=93)
                                             predicate:ca_address_sk is not null
diff --git a/ql/src/test/results/clientpositive/perf/tez/query23.q.out b/ql/src/test/results/clientpositive/perf/tez/query23.q.out
index e60051b..edde953 100644
--- a/ql/src/test/results/clientpositive/perf/tez/query23.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/query23.q.out
@@ -173,25 +173,25 @@ Stage-0
                           PartitionCols:_col0
                           Group By Operator [GBY_501] (rows=62562 width=4)
                             Output:["_col0"],keys:_col1
-                            Select Operator [SEL_500] (rows=183358851 width=290)
+                            Select Operator [SEL_500] (rows=183358851 width=220)
                               Output:["_col1"]
-                              Filter Operator [FIL_499] (rows=183358851 width=290)
+                              Filter Operator [FIL_499] (rows=183358851 width=220)
                                 predicate:(_col3 > 4L)
-                                Select Operator [SEL_498] (rows=550076554 width=290)
+                                Select Operator [SEL_498] (rows=550076554 width=220)
                                   Output:["_col1","_col3"]
-                                  Group By Operator [GBY_497] (rows=550076554 width=290)
+                                  Group By Operator [GBY_497] (rows=550076554 width=220)
                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                   <-Reducer 15 [SIMPLE_EDGE]
                                     SHUFFLE [RS_23]
                                       PartitionCols:_col0
-                                      Group By Operator [GBY_22] (rows=550076554 width=290)
+                                      Group By Operator [GBY_22] (rows=550076554 width=220)
                                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col3, _col5
-                                        Merge Join Operator [MERGEJOIN_442] (rows=550076554 width=282)
+                                        Merge Join Operator [MERGEJOIN_442] (rows=550076554 width=212)
                                           Conds:RS_18._col1=RS_496._col0(Inner),Output:["_col3","_col4","_col5"]
                                         <-Map 19 [SIMPLE_EDGE] vectorized
                                           SHUFFLE [RS_496]
                                             PartitionCols:_col0
-                                            Select Operator [SEL_495] (rows=462000 width=188)
+                                            Select Operator [SEL_495] (rows=462000 width=118)
                                               Output:["_col0","_col1"]
                                               Filter Operator [FIL_494] (rows=462000 width=188)
                                                 predicate:i_item_sk is not null
diff --git a/ql/src/test/results/clientpositive/perf/tez/query79.q.out b/ql/src/test/results/clientpositive/perf/tez/query79.q.out
index f7c8e2f..83b29ee 100644
--- a/ql/src/test/results/clientpositive/perf/tez/query79.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/query79.q.out
@@ -72,17 +72,17 @@ Stage-0
     Stage-1
       Reducer 3 vectorized
       File Output Operator [FS_127]
-        Limit [LIM_126] (rows=100 width=776)
+        Limit [LIM_126] (rows=100 width=592)
           Number of rows:100
-          Select Operator [SEL_125] (rows=479121995 width=776)
+          Select Operator [SEL_125] (rows=479121995 width=592)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           <-Reducer 2 [SIMPLE_EDGE]
             SHUFFLE [RS_33]
-              Select Operator [SEL_32] (rows=479121995 width=776)
+              Select Operator [SEL_32] (rows=479121995 width=592)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                Top N Key Operator [TNK_57] (rows=479121995 width=685)
+                Top N Key Operator [TNK_57] (rows=479121995 width=593)
                   keys:_col2, _col1, substr(_col5, 1, 30), _col7,top n:100
-                  Merge Join Operator [MERGEJOIN_101] (rows=479121995 width=685)
+                  Merge Join Operator [MERGEJOIN_101] (rows=479121995 width=593)
                     Conds:RS_104._col0=RS_124._col1(Inner),Output:["_col1","_col2","_col3","_col5","_col6","_col7","_col8"]
                   <-Map 1 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_104]
@@ -96,7 +96,7 @@ Stage-0
                   <-Reducer 8 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_124]
                       PartitionCols:_col1
-                      Select Operator [SEL_123] (rows=479121995 width=508)
+                      Select Operator [SEL_123] (rows=479121995 width=416)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                         Group By Operator [GBY_122] (rows=479121995 width=328)
                           Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
diff --git a/ql/src/test/results/clientpositive/perf/tez/query8.q.out b/ql/src/test/results/clientpositive/perf/tez/query8.q.out
index c72498b..27420bd 100644
--- a/ql/src/test/results/clientpositive/perf/tez/query8.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/query8.q.out
@@ -271,7 +271,7 @@ Stage-0
                           <-Map 18 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_149]
                               PartitionCols:_col2
-                              Select Operator [SEL_148] (rows=1704 width=276)
+                              Select Operator [SEL_148] (rows=1704 width=178)
                                 Output:["_col0","_col1","_col2"]
                                 Filter Operator [FIL_147] (rows=1704 width=181)
                                   predicate:(s_store_sk is not null and substr(s_zip, 1, 2) is not null)
@@ -280,24 +280,24 @@ Stage-0
                           <-Reducer 11 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_146]
                               PartitionCols:_col0
-                              Select Operator [SEL_145] (rows=1 width=184)
+                              Select Operator [SEL_145] (rows=1 width=86)
                                 Output:["_col0"]
-                                Filter Operator [FIL_144] (rows=1 width=192)
+                                Filter Operator [FIL_144] (rows=1 width=96)
                                   predicate:(_col1 = 2L)
-                                  Group By Operator [GBY_143] (rows=5633 width=192)
+                                  Group By Operator [GBY_143] (rows=5633 width=96)
                                     Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
                                   <-Union 10 [SIMPLE_EDGE]
                                     <-Reducer 16 [CONTAINS] vectorized
                                       Reduce Output Operator [RS_175]
                                         PartitionCols:_col0
-                                        Group By Operator [GBY_174] (rows=5633 width=192)
+                                        Group By Operator [GBY_174] (rows=5633 width=96)
                                           Output:["_col0","_col1"],aggregations:["count(_col1)"],keys:_col0
-                                          Group By Operator [GBY_173] (rows=1126 width=192)
+                                          Group By Operator [GBY_173] (rows=1126 width=96)
                                             Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
                                           <-Reducer 15 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_172]
                                               PartitionCols:_col0
-                                              Group By Operator [GBY_171] (rows=1126 width=192)
+                                              Group By Operator [GBY_171] (rows=1126 width=96)
                                                 Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
                                                 Select Operator [SEL_170] (rows=2253 width=97)
                                                   Output:["_col0"]
@@ -333,14 +333,14 @@ Stage-0
                                     <-Reducer 9 [CONTAINS] vectorized
                                       Reduce Output Operator [RS_161]
                                         PartitionCols:_col0
-                                        Group By Operator [GBY_160] (rows=5633 width=192)
+                                        Group By Operator [GBY_160] (rows=5633 width=96)
                                           Output:["_col0","_col1"],aggregations:["count(_col1)"],keys:_col0
-                                          Group By Operator [GBY_159] (rows=10141 width=192)
+                                          Group By Operator [GBY_159] (rows=10141 width=96)
                                             Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
                                           <-Map 8 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_158]
                                               PartitionCols:_col0
-                                              Group By Operator [GBY_157] (rows=141974 width=192)
+                                              Group By Operator [GBY_157] (rows=141974 width=96)
                                                 Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
                                                 Select Operator [SEL_156] (rows=20000000 width=89)
                                                   Output:["_col0"]
diff --git a/ql/src/test/results/clientpositive/perf/tez/query85.q.out b/ql/src/test/results/clientpositive/perf/tez/query85.q.out
index 6e1a562..94ec2f9 100644
--- a/ql/src/test/results/clientpositive/perf/tez/query85.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/query85.q.out
@@ -201,13 +201,13 @@ Stage-0
     Stage-1
       Reducer 6 vectorized
       File Output Operator [FS_239]
-        Limit [LIM_238] (rows=72 width=832)
+        Limit [LIM_238] (rows=72 width=656)
           Number of rows:100
-          Select Operator [SEL_237] (rows=72 width=832)
+          Select Operator [SEL_237] (rows=72 width=656)
             Output:["_col0","_col1","_col2","_col3"]
           <-Reducer 5 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_236]
-              Select Operator [SEL_235] (rows=72 width=832)
+              Select Operator [SEL_235] (rows=72 width=656)
                 Output:["_col4","_col5","_col6","_col7"]
                 Top N Key Operator [TNK_234] (rows=72 width=353)
                   keys:substr(_col0, 1, 20), (UDFToDouble(_col1) / _col2), (_col3 / _col4), (_col5 / _col6),top n:100
diff --git a/ql/src/test/results/clientpositive/perf/tez/query99.q.out b/ql/src/test/results/clientpositive/perf/tez/query99.q.out
index d24d5cc..87e2713 100644
--- a/ql/src/test/results/clientpositive/perf/tez/query99.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/query99.q.out
@@ -96,24 +96,24 @@ Stage-0
     Stage-1
       Reducer 7 vectorized
       File Output Operator [FS_131]
-        Limit [LIM_130] (rows=100 width=590)
+        Limit [LIM_130] (rows=100 width=420)
           Number of rows:100
-          Select Operator [SEL_129] (rows=3920468 width=590)
+          Select Operator [SEL_129] (rows=3920468 width=420)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
           <-Reducer 6 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_128]
-              Select Operator [SEL_127] (rows=3920468 width=590)
+              Select Operator [SEL_127] (rows=3920468 width=420)
                 Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
-                Group By Operator [GBY_126] (rows=3920468 width=406)
+                Group By Operator [GBY_126] (rows=3920468 width=321)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)"],keys:KEY._col0, KEY._col1, KEY._col2
                 <-Reducer 5 [SIMPLE_EDGE]
                   SHUFFLE [RS_29]
                     PartitionCols:_col0, _col1, _col2
-                    Group By Operator [GBY_28] (rows=7840936 width=406)
+                    Group By Operator [GBY_28] (rows=7840936 width=321)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col4)","sum(_col5)","sum(_col6)","sum(_col7)","sum(_col8)"],keys:_col13, _col15, _col11
-                      Top N Key Operator [TNK_59] (rows=15681873 width=386)
+                      Top N Key Operator [TNK_59] (rows=15681873 width=301)
                         keys:_col13, _col15, _col11,top n:100
-                        Merge Join Operator [MERGEJOIN_105] (rows=15681873 width=386)
+                        Merge Join Operator [MERGEJOIN_105] (rows=15681873 width=301)
                           Conds:RS_24._col2=RS_108._col0(Inner),Output:["_col4","_col5","_col6","_col7","_col8","_col11","_col13","_col15"]
                         <-Map 11 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_108]
@@ -127,12 +127,12 @@ Stage-0
                         <-Reducer 4 [SIMPLE_EDGE]
                           SHUFFLE [RS_24]
                             PartitionCols:_col2
-                            Merge Join Operator [MERGEJOIN_104] (rows=282273729 width=305)
+                            Merge Join Operator [MERGEJOIN_104] (rows=282273729 width=220)
                               Conds:RS_21._col3=RS_125._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col7","_col8","_col11","_col13"]
                             <-Map 10 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_125]
                                 PartitionCols:_col0
-                                Select Operator [SEL_124] (rows=27 width=188)
+                                Select Operator [SEL_124] (rows=27 width=103)
                                   Output:["_col0","_col1"]
                                   Filter Operator [FIL_123] (rows=27 width=104)
                                     predicate:w_warehouse_sk is not null
diff --git a/ql/src/test/results/clientpositive/spark/union17.q.out b/ql/src/test/results/clientpositive/spark/union17.q.out
index c645207..91939f2 100644
--- a/ql/src/test/results/clientpositive/spark/union17.q.out
+++ b/ql/src/test/results/clientpositive/spark/union17.q.out
@@ -85,13 +85,13 @@ STAGE PLANS:
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 1 Data size: 280 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 1 Data size: 275 Basic stats: COMPLETE Column stats: PARTIAL
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
                         null sort order: zz
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 1 Data size: 280 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 1 Data size: 275 Basic stats: COMPLETE Column stats: PARTIAL
         Map 7 
             Map Operator Tree:
                 TableScan
@@ -107,13 +107,13 @@ STAGE PLANS:
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1 Data size: 464 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 1 Data size: 459 Basic stats: COMPLETE Column stats: PARTIAL
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                         null sort order: zzz
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                        Statistics: Num rows: 1 Data size: 464 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 1 Data size: 459 Basic stats: COMPLETE Column stats: PARTIAL
         Reducer 3 
             Reduce Operator Tree:
               Group By Operator
@@ -171,13 +171,13 @@ STAGE PLANS:
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 1 Data size: 280 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 1 Data size: 275 Basic stats: COMPLETE Column stats: PARTIAL
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string)
                       null sort order: zz
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1 Data size: 280 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 1 Data size: 275 Basic stats: COMPLETE Column stats: PARTIAL
         Reducer 9 
             Reduce Operator Tree:
               Group By Operator
@@ -195,13 +195,13 @@ STAGE PLANS:
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 1 Data size: 464 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 1 Data size: 459 Basic stats: COMPLETE Column stats: PARTIAL
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                       null sort order: zzz
                       sort order: +++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                      Statistics: Num rows: 1 Data size: 464 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 1 Data size: 459 Basic stats: COMPLETE Column stats: PARTIAL
 
   Stage: Stage-0
     Move Operator
diff --git a/ql/src/test/results/clientpositive/union17.q.out b/ql/src/test/results/clientpositive/union17.q.out
index 480befa..45b0862 100644
--- a/ql/src/test/results/clientpositive/union17.q.out
+++ b/ql/src/test/results/clientpositive/union17.q.out
@@ -95,20 +95,20 @@ STAGE PLANS:
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 250 Data size: 70000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 250 Data size: 68750 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 70000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 68750 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: count(DISTINCT substr(_col1, 5))
                 keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 501 Data size: 232464 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 501 Data size: 229959 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -130,20 +130,20 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 250 Data size: 70000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 68750 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     null sort order: zz
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 250 Data size: 70000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 250 Data size: 68750 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(DISTINCT substr(_col1, 5))
                   keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 501 Data size: 232464 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 501 Data size: 229959 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     table:
@@ -245,7 +245,7 @@ STAGE PLANS:
               null sort order: zzz
               sort order: +++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-              Statistics: Num rows: 501 Data size: 232464 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 501 Data size: 229959 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
diff --git a/ql/src/test/results/clientpositive/vector_case_when_1.q.out b/ql/src/test/results/clientpositive/vector_case_when_1.q.out
index bedde47..e7117e3 100644
--- a/ql/src/test/results/clientpositive/vector_case_when_1.q.out
+++ b/ql/src/test/results/clientpositive/vector_case_when_1.q.out
@@ -213,13 +213,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [4, 21, 26, 30, 34, 38, 42, 44, 46, 48, 50, 52, 54, 58, 61, 64, 67]
                   selectExpressions: VectorUDFAdaptor(CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, LongColLessLongScalar(col 4:int, val 100) -> 20:boolean) -> 21:string, Ve [...]
-              Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -546,13 +546,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [4, 24, 33, 40, 44, 48, 52, 54, 56, 58, 60, 62, 64, 68, 71, 74, 77]
                   selectExpressions: IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 22:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 21:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprStringScalarStringScalar( [...]
-              Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -879,13 +879,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [4, 27, 39, 48, 52, 57, 62, 64, 66, 71, 76, 78, 80, 84, 87, 90, 93]
                   selectExpressions: IfExprColumnCondExpr(col 17:boolean, col 18:stringcol 26:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, ConstantVectorExpression(val Single) -> 18:string, IfExprColumnCondExpr(col 19:boolean, col 20:stringcol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 19:boolean, ConstantVectorExpression(val Two) -> 20:string, IfExprColumnCondExpr(col 21:boolean, col 22:stringcol 24:string)(children: LongColLessLongScal [...]
-              Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 101 Data size: 118604 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/vector_case_when_2.q.out b/ql/src/test/results/clientpositive/vector_case_when_2.q.out
index 288e6f0..a7b46fd 100644
--- a/ql/src/test/results/clientpositive/vector_case_when_2.q.out
+++ b/ql/src/test/results/clientpositive/vector_case_when_2.q.out
@@ -145,7 +145,7 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [1, 3, 9, 14, 18, 22, 24, 27, 32, 38, 2]
                   selectExpressions: VectorUDFAdaptor(CASE WHEN ((ctimestamp2 <= TIMESTAMP'1800-12-31 00:00:00')) THEN ('1800s or Earlier') WHEN ((ctimestamp2 < TIMESTAMP'1900-01-01 00:00:00')) THEN ('1900s') WHEN (ctimestamp2 BETWEEN TIMESTAMP'2006-01-01 00:00:00' AND TIMESTAMP'2010-12-31 23:59:59.999999999') THEN ('Late 2000s') WHEN ((ctimestamp2 <= TIMESTAMP'2015-12-31 23:59:59.999999999')) THEN ('Early 2010s') ELSE ('Unknown') END)(children: TimestampColLessEqualTimestampScalar(col 3 [...]
-              Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: timestamp), _col10 (type: string), _col1 (type: timestamp)
                 null sort order: zzz
@@ -155,7 +155,7 @@ STAGE PLANS:
                     native: false
                     nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                     nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
                 value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int), _col8 (type: int), _col9 (type: date)
       Execution mode: vectorized
       Map Vectorization:
@@ -181,10 +181,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: timestamp), KEY.reducesinkkey2 (type: timestamp), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: date)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-          Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -409,7 +409,7 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [1, 3, 12, 21, 28, 32, 34, 37, 42, 48, 2]
                   selectExpressions: IfExprStringScalarStringGroupColumn(col 5:boolean, val 1800s or Earliercol 11:string)(children: TimestampColLessEqualTimestampScalar(col 3:timestamp, val 1800-12-31 00:00:00) -> 5:boolean, IfExprStringScalarStringGroupColumn(col 6:boolean, val 1900scol 10:string)(children: TimestampColLessTimestampScalar(col 3:timestamp, val 1900-01-01 00:00:00) -> 6:boolean, IfExprStringScalarStringGroupColumn(col 7:boolean, val Late 2000scol 9:string)(children: Time [...]
-              Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: timestamp), _col10 (type: string), _col1 (type: timestamp)
                 null sort order: zzz
@@ -419,7 +419,7 @@ STAGE PLANS:
                     native: false
                     nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                     nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
                 value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int), _col8 (type: int), _col9 (type: date)
       Execution mode: vectorized
       Map Vectorization:
@@ -445,10 +445,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: timestamp), KEY.reducesinkkey2 (type: timestamp), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: date)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-          Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -673,7 +673,7 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [1, 3, 15, 27, 36, 40, 42, 45, 50, 56, 2]
                   selectExpressions: IfExprColumnCondExpr(col 5:boolean, col 6:stringcol 14:string)(children: TimestampColLessEqualTimestampScalar(col 3:timestamp, val 1800-12-31 00:00:00) -> 5:boolean, ConstantVectorExpression(val 1800s or Earlier) -> 6:string, IfExprColumnCondExpr(col 7:boolean, col 8:stringcol 13:string)(children: TimestampColLessTimestampScalar(col 3:timestamp, val 1900-01-01 00:00:00) -> 7:boolean, ConstantVectorExpression(val 1900s) -> 8:string, IfExprColumnCondExp [...]
-              Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: timestamp), _col10 (type: string), _col1 (type: timestamp)
                 null sort order: zzz
@@ -683,7 +683,7 @@ STAGE PLANS:
                     native: false
                     nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                     nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                Statistics: Num rows: 51 Data size: 50745 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 51 Data size: 50345 Basic stats: COMPLETE Column stats: COMPLETE
                 value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int), _col8 (type: int), _col9 (type: date)
       Execution mode: vectorized
       Map Vectorization:
@@ -709,10 +709,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: timestamp), KEY.reducesinkkey2 (type: timestamp), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: date)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-          Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 51 Data size: 45084 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 51 Data size: 44684 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/vector_groupby4.q.out b/ql/src/test/results/clientpositive/vector_groupby4.q.out
index 925b96c..8ea182a 100644
--- a/ql/src/test/results/clientpositive/vector_groupby4.q.out
+++ b/ql/src/test/results/clientpositive/vector_groupby4.q.out
@@ -90,7 +90,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: partial1
           outputColumnNames: _col0
-          Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -114,7 +114,7 @@ STAGE PLANS:
                   native: false
                   nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                   nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Map Vectorization:
           enabled: true
@@ -134,10 +134,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: final
           outputColumnNames: _col0
-          Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
@@ -146,7 +146,7 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: string)
             outputColumnNames: c1
-            Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -187,7 +187,7 @@ STAGE PLANS:
                   native: false
                   nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                   nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 316 Data size: 26860 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: c1 (type: string)
       Execution mode: vectorized
       Map Vectorization:
diff --git a/ql/src/test/results/clientpositive/vector_groupby6.q.out b/ql/src/test/results/clientpositive/vector_groupby6.q.out
index b478656..2cba267 100644
--- a/ql/src/test/results/clientpositive/vector_groupby6.q.out
+++ b/ql/src/test/results/clientpositive/vector_groupby6.q.out
@@ -90,7 +90,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: partial1
           outputColumnNames: _col0
-          Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -114,7 +114,7 @@ STAGE PLANS:
                   native: false
                   nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                   nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 42500 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Map Vectorization:
           enabled: true
@@ -134,10 +134,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: final
           outputColumnNames: _col0
-          Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
@@ -146,7 +146,7 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: string)
             outputColumnNames: c1
-            Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -187,7 +187,7 @@ STAGE PLANS:
                   native: false
                   nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                   nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 307 Data size: 56488 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 307 Data size: 26095 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: c1 (type: string)
       Execution mode: vectorized
       Map Vectorization:
diff --git a/ql/src/test/results/clientpositive/vector_if_expr.q.out b/ql/src/test/results/clientpositive/vector_if_expr.q.out
index 58c2e1e..292e8c6 100644
--- a/ql/src/test/results/clientpositive/vector_if_expr.q.out
+++ b/ql/src/test/results/clientpositive/vector_if_expr.q.out
@@ -41,7 +41,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [10, 13]
                     selectExpressions: IfExprStringScalarStringScalar(col 10:boolean, val first, val second) -> 13:string
-                Statistics: Num rows: 4587 Data size: 857712 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4587 Data size: 426534 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: boolean)
                   null sort order: z
@@ -51,7 +51,7 @@ STAGE PLANS:
                       native: false
                       nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                       nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                  Statistics: Num rows: 4587 Data size: 857712 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4587 Data size: 426534 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string)
       Execution mode: vectorized
       Map Vectorization:
@@ -71,10 +71,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: boolean), VALUE._col0 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 4587 Data size: 857712 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 4587 Data size: 426534 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 4587 Data size: 857712 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 4587 Data size: 426534 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/vector_nvl.q.out b/ql/src/test/results/clientpositive/vector_nvl.q.out
index 43ca0ec..26bae3f 100644
--- a/ql/src/test/results/clientpositive/vector_nvl.q.out
+++ b/ql/src/test/results/clientpositive/vector_nvl.q.out
@@ -140,19 +140,19 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [4, 14]
                   selectExpressions: VectorCoalesce(columns [4, 13])(children: col 4:float, ConstantVectorExpression(val 1.0) -> 13:float) -> 14:float
-              Statistics: Num rows: 12288 Data size: 85848 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 73392 Basic stats: COMPLETE Column stats: COMPLETE
               Limit
                 Number of rows: 10
                 Limit Vectorization:
                     className: VectorLimitOperator
                     native: true
-                Statistics: Num rows: 10 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 10 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 10 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 10 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/vectorization_multi_value.q.out b/ql/src/test/results/clientpositive/vectorization_multi_value.q.out
index 2fec50d..b4507fb 100644
--- a/ql/src/test/results/clientpositive/vectorization_multi_value.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_multi_value.q.out
@@ -64,13 +64,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [4]
                   selectExpressions: IfExprCondExprNull(col 2:boolean, col 3:map<string,string>, null)(children: ConstantVectorExpression(val 1) -> 2:boolean, VectorUDFAdaptor(map('a':'b')) -> 3:map<string,string>) -> 4:map<string,string>
-              Statistics: Num rows: 3 Data size: 978 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 326 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 3 Data size: 978 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 326 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -143,13 +143,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [5]
                   selectExpressions: IfExprCondExprNull(col 2:boolean, col 4:map<string,map<string,string>>, null)(children: ConstantVectorExpression(val 1) -> 2:boolean, VectorUDFAdaptor(map('a':map('b':'c')))(children: VectorUDFAdaptor(map('b':'c')) -> 3:map<string,string>) -> 4:map<string,map<string,string>>) -> 5:map<string,map<string,string>>
-              Statistics: Num rows: 3 Data size: 723 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 241 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 3 Data size: 723 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 241 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -222,13 +222,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [4]
                   selectExpressions: IfExprCondExprNull(col 2:boolean, col 3:map<string,string>, null)(children: ConstantVectorExpression(val 1) -> 2:boolean, VectorUDFAdaptor(map('a':a)) -> 3:map<string,string>) -> 4:map<string,string>
-              Statistics: Num rows: 3 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 3 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -301,13 +301,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [5]
                   selectExpressions: IfExprCondExprNull(col 2:boolean, col 4:map<string,map<string,string>>, null)(children: ConstantVectorExpression(val 1) -> 2:boolean, VectorUDFAdaptor(map('a':map('b':a)))(children: VectorUDFAdaptor(map('b':a)) -> 3:map<string,string>) -> 4:map<string,map<string,string>>) -> 5:map<string,map<string,string>>
-              Statistics: Num rows: 3 Data size: 4968 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 3 Data size: 4968 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -380,13 +380,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [4]
                   selectExpressions: IfExprCondExprNull(col 2:boolean, col 3:array<string>, null)(children: ConstantVectorExpression(val 1) -> 2:boolean, VectorUDFAdaptor(array('a','b')) -> 3:array<string>) -> 4:array<string>
-              Statistics: Num rows: 3 Data size: 1224 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 3 Data size: 1224 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -459,13 +459,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [6]
                   selectExpressions: IfExprCondExprNull(col 2:boolean, col 5:array<array<string>>, null)(children: ConstantVectorExpression(val 1) -> 2:boolean, VectorUDFAdaptor(array(array('a','b'),array('c','d')))(children: VectorUDFAdaptor(array('a','b')) -> 3:array<string>, VectorUDFAdaptor(array('c','d')) -> 4:array<string>) -> 5:array<array<string>>) -> 6:array<array<string>>
-              Statistics: Num rows: 3 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 3 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -538,13 +538,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [4]
                   selectExpressions: IfExprCondExprNull(col 2:boolean, col 3:array<string>, null)(children: ConstantVectorExpression(val 1) -> 2:boolean, VectorUDFAdaptor(array('a',a)) -> 3:array<string>) -> 4:array<string>
-              Statistics: Num rows: 3 Data size: 5760 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 3 Data size: 5760 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -617,13 +617,13 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [6]
                   selectExpressions: IfExprCondExprNull(col 2:boolean, col 5:array<array<string>>, null)(children: ConstantVectorExpression(val 1) -> 2:boolean, VectorUDFAdaptor(array(array('a',a),array('b','c')))(children: VectorUDFAdaptor(array('a',a)) -> 3:array<string>, VectorUDFAdaptor(array('b','c')) -> 4:array<string>) -> 5:array<array<string>>) -> 6:array<array<string>>
-              Statistics: Num rows: 3 Data size: 57840 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 19280 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 File Sink Vectorization:
                     className: VectorFileSinkOperator
                     native: false
-                Statistics: Num rows: 3 Data size: 57840 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 19280 Basic stats: COMPLETE Column stats: COMPLETE
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/vectorized_string_funcs.q.out b/ql/src/test/results/clientpositive/vectorized_string_funcs.q.out
index 75da191..2c0504a 100644
--- a/ql/src/test/results/clientpositive/vectorized_string_funcs.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_string_funcs.q.out
@@ -68,10 +68,10 @@ STAGE PLANS:
               Select Operator
                 expressions: substr(cstring1, 1, 2) (type: string), substr(cstring1, 2) (type: string), lower(cstring1) (type: string), upper(cstring1) (type: string), upper(cstring1) (type: string), length(cstring1) (type: int), trim(cstring1) (type: string), ltrim(cstring1) (type: string), rtrim(cstring1) (type: string), concat(cstring1, cstring2) (type: string), concat('>', cstring1) (type: string), concat(cstring1, '<') (type: string), concat(substr(cstring1, 1, 2), substr(cstring2,  [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                Statistics: Num rows: 1024 Data size: 2265088 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1024 Data size: 2024426 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1024 Data size: 2265088 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1024 Data size: 2024426 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
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 907edb6..c94eb90 100644
--- a/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
@@ -268,7 +268,7 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 0, 1, 3, 13, 14, 15, 16, 17]
                   selectExpressions: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 6:int, VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 7:int, VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 8:int, VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 9:int, VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 10:int, VectorUDFMinuteTimestamp(col 1:timestamp, fie [...]
-              Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 52 Data size: 11916 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: bigint)
                 null sort order: z
@@ -278,7 +278,7 @@ STAGE PLANS:
                     native: false
                     nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                     nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 52 Data size: 11916 Basic stats: COMPLETE Column stats: COMPLETE
                 value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int), _col9 (type: boolean), _col10 (type: timestamp), _col11 (type: timestamp), _col12 (type: timestamp), _col13 (type: timestamp), _col14 (type: timestamp), _col15 (type: timestamp), _col16 (type: timestamp)
       Execution mode: vectorized
       Map Vectorization:
@@ -298,10 +298,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: boolean), VALUE._col8 (type: timestamp), VALUE._col9 (type: timestamp), VALUE._col10 (type: timestamp), VALUE._col11 (type: timestamp), VALUE._col12 (type: timestamp), VALUE._col13 (type: timestamp), VALUE._col14 (t [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
-          Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 52 Data size: 11916 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 52 Data size: 11916 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java
index d27323a..ec1f169 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java
@@ -22,6 +22,7 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileReader;
 import java.io.IOException;
+import java.nio.file.NoSuchFileException;
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
@@ -132,9 +133,11 @@ public class MetaStoreSchemaInfo implements IMetaStoreSchemaInfo {
     String initScriptName = INIT_FILE_PREFIX + toVersion + "." +
         dbType + SQL_FILE_EXTENSION;
     // check if the file exists
-    if (!(new File(getMetaStoreScriptDir() + File.separatorChar +
-          initScriptName).exists())) {
-      throw new HiveMetaException("Unknown version specified for initialization: " + toVersion);
+    File file = new File(getMetaStoreScriptDir() + File.separatorChar +
+          initScriptName);
+    if (!file.exists()) {
+      throw new HiveMetaException("Unknown version specified for initialization: " + toVersion,
+          new NoSuchFileException(file.getAbsolutePath()));
     }
     return initScriptName;
   }