You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2019/11/06 15:55:21 UTC

[hive] branch master updated: HIVE-22311: Propagate min/max column values from statistics to the optimizer for timestamp type (Jesus Camacho Rodriguez, reviewed by Miklos Gergely)

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

jcamacho 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 4cfb7a0  HIVE-22311: Propagate min/max column values from statistics to the optimizer for timestamp type (Jesus Camacho Rodriguez, reviewed by Miklos Gergely)
4cfb7a0 is described below

commit 4cfb7a02eabbd63f9ddaa05a4f2f1dcc2ad76f1e
Author: Jesus Camacho Rodriguez <jc...@apache.org>
AuthorDate: Tue Sep 24 13:34:59 2019 -0700

    HIVE-22311: Propagate min/max column values from statistics to the optimizer for timestamp type (Jesus Camacho Rodriguez, reviewed by Miklos Gergely)
    
    Close apache/hive#821
---
 .../hadoop/hive/ql/exec/ColumnStatsUpdateTask.java |  36 +-
 .../metadata/formatting/JsonMetaDataFormatter.java |  84 ++-
 .../metadata/formatting/MetaDataFormatUtils.java   |  20 +
 .../stats/annotation/StatsRulesProcFactory.java    |  24 +-
 .../ql/stats/ColumnStatisticsObjTranslator.java    |  35 +
 .../apache/hadoop/hive/ql/stats/StatsUtils.java    |  22 +-
 .../ql/udf/generic/GenericUDAFComputeStats.java    |  83 ++-
 .../queries/clientpositive/timestamp_comparison3.q |  32 +
 .../clientpositive/alter_table_update_status.q.out |  12 +-
 ...ter_table_update_status_disable_bitvector.q.out |  12 +-
 .../beeline/desc_table_formatted.q.out             |   6 +-
 .../clientpositive/confirm_initial_tbl_stats.q.out |   4 +-
 .../results/clientpositive/constprog_type.q.out    |  10 +-
 .../clientpositive/desc_table_formatted.q.out      |   6 +-
 .../druid/druidmini_dynamic_partition.q.out        |  42 +-
 ql/src/test/results/clientpositive/foldts.q.out    |   8 +-
 .../clientpositive/interval_arithmetic.q.out       |  12 +-
 .../clientpositive/llap/explainuser_4.q.out        |  16 +-
 .../results/clientpositive/llap/orc_llap.q.out     |  12 +-
 .../clientpositive/llap/orc_llap_nonvector.q.out   |   8 +-
 .../results/clientpositive/llap/orc_merge5.q.out   |  20 +-
 .../results/clientpositive/llap/orc_merge6.q.out   |   8 +-
 .../results/clientpositive/llap/orc_merge7.q.out   |   8 +-
 .../clientpositive/llap/orc_merge_incompat1.q.out  |  10 +-
 .../clientpositive/llap/orc_merge_incompat2.q.out  |   4 +-
 .../llap/tez_dynpart_hashjoin_2.q.out              |  60 +-
 .../llap/tez_dynpart_hashjoin_3.q.out              |  12 +-
 .../llap/tez_vector_dynpart_hashjoin_2.q.out       |  60 +-
 .../clientpositive/llap/vector_aggregate_9.q.out   |   4 +-
 .../clientpositive/llap/vector_coalesce.q.out      |   2 +-
 .../clientpositive/llap/vector_complex_join.q.out  |  10 +-
 .../clientpositive/llap/vector_decimal_cast.q.out  |  32 +-
 .../llap/vector_interval_mapjoin.q.out             |  20 +-
 .../clientpositive/llap/vector_outer_join1.q.out   |  16 +-
 .../llap/vector_reuse_scratchcols.q.out            |  12 +-
 .../clientpositive/llap/vectorization_0.q.out      |   6 +-
 .../clientpositive/llap/vectorization_10.q.out     |   8 +-
 .../clientpositive/llap/vectorization_11.q.out     |   8 +-
 .../clientpositive/llap/vectorization_12.q.out     |  20 +-
 .../clientpositive/llap/vectorization_13.q.out     |  48 +-
 .../clientpositive/llap/vectorization_14.q.out     |  22 +-
 .../clientpositive/llap/vectorization_15.q.out     |  22 +-
 .../clientpositive/llap/vectorization_16.q.out     |  18 +-
 .../clientpositive/llap/vectorization_17.q.out     |  12 +-
 .../clientpositive/llap/vectorization_2.q.out      |   6 +-
 .../clientpositive/llap/vectorization_3.q.out      |   6 +-
 .../clientpositive/llap/vectorization_5.q.out      |   6 +-
 .../clientpositive/llap/vectorization_7.q.out      |  28 +-
 .../clientpositive/llap/vectorization_8.q.out      |  28 +-
 .../clientpositive/llap/vectorization_9.q.out      |  18 +-
 .../llap/vectorization_short_regress.q.out         | 104 +--
 .../clientpositive/llap/vectorized_casts.q.out     |   8 +-
 .../llap/vectorized_timestamp_funcs.q.out          |  24 +-
 .../llap/vectorized_timestamp_ints_casts.q.out     |  16 +-
 .../test/results/clientpositive/orc_merge5.q.out   |  20 +-
 .../test/results/clientpositive/orc_merge6.q.out   |   8 +-
 .../clientpositive/orc_merge_incompat1.q.out       |  10 +-
 .../clientpositive/orc_merge_incompat2.q.out       |   4 +-
 .../clientpositive/parquet_vectorization_0.q.out   |   6 +-
 .../clientpositive/parquet_vectorization_10.q.out  |   8 +-
 .../clientpositive/parquet_vectorization_11.q.out  |   8 +-
 .../clientpositive/parquet_vectorization_12.q.out  |  20 +-
 .../clientpositive/parquet_vectorization_13.q.out  |  44 +-
 .../clientpositive/parquet_vectorization_14.q.out  |  20 +-
 .../clientpositive/parquet_vectorization_15.q.out  |  20 +-
 .../clientpositive/parquet_vectorization_16.q.out  |  16 +-
 .../clientpositive/parquet_vectorization_17.q.out  |  12 +-
 .../clientpositive/parquet_vectorization_2.q.out   |   6 +-
 .../clientpositive/parquet_vectorization_3.q.out   |   6 +-
 .../clientpositive/parquet_vectorization_5.q.out   |   6 +-
 .../clientpositive/parquet_vectorization_7.q.out   |  28 +-
 .../clientpositive/parquet_vectorization_8.q.out   |  28 +-
 .../clientpositive/parquet_vectorization_9.q.out   |  16 +-
 .../clientpositive/partitioned_table_stats.q.out   |  12 +-
 .../clientpositive/tez/explainanalyze_3.q.out      |   2 +-
 .../clientpositive/tez/explainanalyze_4.q.out      |  16 +-
 .../results/clientpositive/tez/explainuser_3.q.out |   4 +-
 .../clientpositive/timestamp_comparison3.q.out     | 212 ++++++
 .../clientpositive/timestamp_ints_casts.q.out      |  16 +-
 .../clientpositive/vector_aggregate_9.q.out        |   4 +-
 .../results/clientpositive/vector_coalesce.q.out   |   2 +-
 .../clientpositive/vector_decimal_cast.q.out       |  32 +-
 .../clientpositive/vector_empty_where.q.out        |   6 +-
 .../clientpositive/vector_interval_mapjoin.q.out   |  18 +-
 .../vector_non_constant_in_expr.q.out              |   8 +-
 .../clientpositive/vector_outer_join1.q.out        |  14 +-
 .../clientpositive/vector_outer_join3.q.out        |   6 +-
 .../clientpositive/vector_outer_join4.q.out        |   6 +-
 .../results/clientpositive/vector_udf_trunc.q.out  |   6 +-
 .../results/clientpositive/vectorization_10.q.out  |   8 +-
 .../results/clientpositive/vectorization_11.q.out  |   8 +-
 .../results/clientpositive/vectorization_12.q.out  |  20 +-
 .../results/clientpositive/vectorization_13.q.out  |  44 +-
 .../results/clientpositive/vectorization_14.q.out  |  20 +-
 .../results/clientpositive/vectorization_15.q.out  |  20 +-
 .../results/clientpositive/vectorization_16.q.out  |  16 +-
 .../results/clientpositive/vectorization_17.q.out  |  12 +-
 .../results/clientpositive/vectorization_2.q.out   |   6 +-
 .../results/clientpositive/vectorization_3.q.out   |   6 +-
 .../results/clientpositive/vectorization_5.q.out   |   6 +-
 .../results/clientpositive/vectorization_7.q.out   |  28 +-
 .../results/clientpositive/vectorization_8.q.out   |  28 +-
 .../results/clientpositive/vectorization_9.q.out   |  16 +-
 .../results/clientpositive/vectorized_casts.q.out  |   8 +-
 .../vectorized_timestamp_funcs.q.out               |  24 +-
 .../vectorized_timestamp_ints_casts.q.out          |  16 +-
 .../hive/metastore/api/ColumnStatisticsData.java   |  63 +-
 .../hadoop/hive/metastore/api/Timestamp.java       | 387 ++++++++++
 .../metastore/api/TimestampColumnStatsData.java    | 823 +++++++++++++++++++++
 .../src/gen/thrift/gen-php/metastore/Types.php     | 280 +++++++
 .../src/gen/thrift/gen-py/hive_metastore/ttypes.py | 206 +++++-
 .../src/gen/thrift/gen-rb/hive_metastore_types.rb  |  51 +-
 .../src/main/thrift/hive_metastore.thrift          |  15 +-
 .../hadoop/hive/metastore/StatObjectConverter.java | 125 +++-
 .../metastore/columnstats/ColumnsStatsUtils.java   |  17 +
 .../aggr/ColumnStatsAggregatorFactory.java         |   8 +
 .../aggr/TimestampColumnStatsAggregator.java       | 362 +++++++++
 .../cache/TimestampColumnStatsDataInspector.java   | 127 ++++
 .../merge/ColumnStatsMergerFactory.java            |   9 +
 .../merge/TimestampColumnStatsMerger.java          |  94 +++
 .../model/MPartitionColumnStatistics.java          |   8 +
 .../metastore/model/MTableColumnStatistics.java    |   8 +
 122 files changed, 3860 insertions(+), 829 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
index 10a2947..526b57a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
@@ -39,12 +39,14 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.Date;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
+import org.apache.hadoop.hive.metastore.api.Timestamp;
 import org.apache.hadoop.hive.metastore.api.utils.DecimalUtils;
 import org.apache.hadoop.hive.metastore.columnstats.cache.DateColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.DecimalColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.DoubleColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.LongColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.StringColumnStatsDataInspector;
+import org.apache.hadoop.hive.metastore.columnstats.cache.TimestampColumnStatsDataInspector;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
@@ -58,6 +60,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ColumnStatsUpdateWork;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -110,7 +113,7 @@ public class ColumnStatsUpdateTask extends Task<ColumnStatsUpdateWork> {
 
     if (columnType.equalsIgnoreCase("long") || columnType.equalsIgnoreCase("tinyint")
         || columnType.equalsIgnoreCase("smallint") || columnType.equalsIgnoreCase("int")
-        || columnType.equalsIgnoreCase("bigint") || columnType.equalsIgnoreCase("timestamp")) {
+        || columnType.equalsIgnoreCase("bigint")) {
       LongColumnStatsDataInspector longStats = new LongColumnStatsDataInspector();
       longStats.setNumNullsIsSet(false);
       longStats.setNumDVsIsSet(false);
@@ -275,6 +278,26 @@ public class ColumnStatsUpdateTask extends Task<ColumnStatsUpdateWork> {
       }
       statsData.setDateStats(dateStats);
       statsObj.setStatsData(statsData);
+    } else if (columnType.equalsIgnoreCase("timestamp")) {
+      TimestampColumnStatsDataInspector timestampStats = new TimestampColumnStatsDataInspector();
+      Map<String, String> mapProp = work.getMapProp();
+      for (Entry<String, String> entry : mapProp.entrySet()) {
+        String fName = entry.getKey();
+        String value = entry.getValue();
+        if (fName.equals("numNulls")) {
+          timestampStats.setNumNulls(Long.parseLong(value));
+        } else if (fName.equals("numDVs")) {
+          timestampStats.setNumDVs(Long.parseLong(value));
+        } else if (fName.equals("lowValue")) {
+          timestampStats.setLowValue(readTimestampValue(value));
+        } else if (fName.equals("highValue")) {
+          timestampStats.setHighValue(readTimestampValue(value));
+        } else {
+          throw new SemanticException("Unknown stat");
+        }
+      }
+      statsData.setTimestampStats(timestampStats);
+      statsObj.setStatsData(statsData);
     } else {
       throw new SemanticException("Unsupported type");
     }
@@ -375,4 +398,15 @@ public class ColumnStatsUpdateTask extends Task<ColumnStatsUpdateWork> {
       return new Date(Long.parseLong(dateStr));
     }
   }
+
+  private Timestamp readTimestampValue(String timestampStr) {
+    try {
+      TimestampWritableV2 writableVal = new TimestampWritableV2(
+          org.apache.hadoop.hive.common.type.Timestamp.valueOf(timestampStr));
+      return new Timestamp(writableVal.getSeconds());
+    } catch (IllegalArgumentException err) {
+      LOG.debug("Reading timestamp value as seconds since epoch: {}", timestampStr);
+      return new Timestamp(Long.parseLong(timestampStr));
+    }
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
index 0b66b93..bff163f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
@@ -70,6 +70,19 @@ import static org.apache.hadoop.hive.conf.Constants.MATERIALIZED_VIEW_REWRITING_
 public class JsonMetaDataFormatter implements MetaDataFormatter {
   private static final Logger LOG = LoggerFactory.getLogger(JsonMetaDataFormatter.class);
 
+  private static final String COLUMN_NAME = "name";
+  private static final String COLUMN_TYPE = "type";
+  private static final String COLUMN_COMMENT = "comment";
+  private static final String COLUMN_MIN = "min";
+  private static final String COLUMN_MAX = "max";
+  private static final String COLUMN_NUM_NULLS = "numNulls";
+  private static final String COLUMN_NUM_TRUES = "numTrues";
+  private static final String COLUMN_NUM_FALSES = "numFalses";
+  private static final String COLUMN_DISTINCT_COUNT = "distinctCount";
+  private static final String COLUMN_AVG_LENGTH = "avgColLen";
+  private static final String COLUMN_MAX_LENGTH = "maxColLen";
+
+
   /**
    * Convert the map to a JSON string.
    */
@@ -247,96 +260,109 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
 
   private Map<String, Object> createColumnInfo(FieldSchema column, ColumnStatisticsData statistics) {
     Map<String, Object> result = MapBuilder.create()
-        .put("name", column.getName())
-        .put("type", column.getType())
-        .put("comment", column.getComment())
+        .put(COLUMN_NAME, column.getName())
+        .put(COLUMN_TYPE, column.getType())
+        .put(COLUMN_COMMENT, column.getComment())
         .build();
 
     if (statistics != null) {
       if (statistics.isSetBinaryStats()) {
         if (statistics.getBinaryStats().isSetNumNulls()) {
-          result.put("numNulls", statistics.getBinaryStats().getNumNulls());
+          result.put(COLUMN_NUM_NULLS, statistics.getBinaryStats().getNumNulls());
         }
         if (statistics.getBinaryStats().isSetAvgColLen()) {
-          result.put("avgColLen", statistics.getBinaryStats().getAvgColLen());
+          result.put(COLUMN_AVG_LENGTH, statistics.getBinaryStats().getAvgColLen());
         }
         if (statistics.getBinaryStats().isSetMaxColLen()) {
-          result.put("maxColLen", statistics.getBinaryStats().getMaxColLen());
+          result.put(COLUMN_MAX_LENGTH, statistics.getBinaryStats().getMaxColLen());
         }
       } else if (statistics.isSetStringStats()) {
         if (statistics.getStringStats().isSetNumNulls()) {
-          result.put("numNulls", statistics.getStringStats().getNumNulls());
+          result.put(COLUMN_NUM_NULLS, statistics.getStringStats().getNumNulls());
         }
         if (statistics.getStringStats().isSetNumDVs()) {
-          result.put("distinctCount", statistics.getStringStats().getNumDVs());
+          result.put(COLUMN_DISTINCT_COUNT, statistics.getStringStats().getNumDVs());
         }
         if (statistics.getStringStats().isSetAvgColLen()) {
-          result.put("avgColLen", statistics.getStringStats().getAvgColLen());
+          result.put(COLUMN_AVG_LENGTH, statistics.getStringStats().getAvgColLen());
         }
         if (statistics.getStringStats().isSetMaxColLen()) {
-          result.put("maxColLen", statistics.getStringStats().getMaxColLen());
+          result.put(COLUMN_MAX_LENGTH, statistics.getStringStats().getMaxColLen());
         }
       } else if (statistics.isSetBooleanStats()) {
         if (statistics.getBooleanStats().isSetNumNulls()) {
-          result.put("numNulls", statistics.getBooleanStats().getNumNulls());
+          result.put(COLUMN_NUM_NULLS, statistics.getBooleanStats().getNumNulls());
         }
         if (statistics.getBooleanStats().isSetNumTrues()) {
-          result.put("numTrues", statistics.getBooleanStats().getNumTrues());
+          result.put(COLUMN_NUM_TRUES, statistics.getBooleanStats().getNumTrues());
         }
         if (statistics.getBooleanStats().isSetNumFalses()) {
-          result.put("numFalses", statistics.getBooleanStats().getNumFalses());
+          result.put(COLUMN_NUM_FALSES, statistics.getBooleanStats().getNumFalses());
         }
       } else if (statistics.isSetDecimalStats()) {
         if (statistics.getDecimalStats().isSetLowValue()) {
-          result.put("min", MetaDataFormatUtils.convertToString(statistics.getDecimalStats().getLowValue()));
+          result.put(COLUMN_MIN, MetaDataFormatUtils.convertToString(statistics.getDecimalStats().getLowValue()));
         }
         if (statistics.getDecimalStats().isSetHighValue()) {
-          result.put("max", MetaDataFormatUtils.convertToString(statistics.getDecimalStats().getHighValue()));
+          result.put(COLUMN_MAX, MetaDataFormatUtils.convertToString(statistics.getDecimalStats().getHighValue()));
         }
         if (statistics.getDecimalStats().isSetNumNulls()) {
-          result.put("numNulls", statistics.getDecimalStats().getNumNulls());
+          result.put(COLUMN_NUM_NULLS, statistics.getDecimalStats().getNumNulls());
         }
         if (statistics.getDecimalStats().isSetNumDVs()) {
-          result.put("distinctCount", statistics.getDecimalStats().getNumDVs());
+          result.put(COLUMN_DISTINCT_COUNT, statistics.getDecimalStats().getNumDVs());
         }
       } else if (statistics.isSetDoubleStats()) {
         if (statistics.getDoubleStats().isSetLowValue()) {
-          result.put("min", statistics.getDoubleStats().getLowValue());
+          result.put(COLUMN_MIN, statistics.getDoubleStats().getLowValue());
         }
         if (statistics.getDoubleStats().isSetHighValue()) {
-          result.put("max", statistics.getDoubleStats().getHighValue());
+          result.put(COLUMN_MAX, statistics.getDoubleStats().getHighValue());
         }
         if (statistics.getDoubleStats().isSetNumNulls()) {
-          result.put("numNulls", statistics.getDoubleStats().getNumNulls());
+          result.put(COLUMN_NUM_NULLS, statistics.getDoubleStats().getNumNulls());
         }
         if (statistics.getDoubleStats().isSetNumDVs()) {
-          result.put("distinctCount", statistics.getDoubleStats().getNumDVs());
+          result.put(COLUMN_DISTINCT_COUNT, statistics.getDoubleStats().getNumDVs());
         }
       } else if (statistics.isSetLongStats()) {
         if (statistics.getLongStats().isSetLowValue()) {
-          result.put("min", statistics.getLongStats().getLowValue());
+          result.put(COLUMN_MIN, statistics.getLongStats().getLowValue());
         }
         if (statistics.getLongStats().isSetHighValue()) {
-          result.put("max", statistics.getLongStats().getHighValue());
+          result.put(COLUMN_MAX, statistics.getLongStats().getHighValue());
         }
         if (statistics.getLongStats().isSetNumNulls()) {
-          result.put("numNulls", statistics.getLongStats().getNumNulls());
+          result.put(COLUMN_NUM_NULLS, statistics.getLongStats().getNumNulls());
         }
         if (statistics.getLongStats().isSetNumDVs()) {
-          result.put("distinctCount", statistics.getLongStats().getNumDVs());
+          result.put(COLUMN_DISTINCT_COUNT, statistics.getLongStats().getNumDVs());
         }
       } else if (statistics.isSetDateStats()) {
         if (statistics.getDateStats().isSetLowValue()) {
-          result.put("min", MetaDataFormatUtils.convertToString(statistics.getDateStats().getLowValue()));
+          result.put(COLUMN_MIN, MetaDataFormatUtils.convertToString(statistics.getDateStats().getLowValue()));
         }
         if (statistics.getDateStats().isSetHighValue()) {
-          result.put("max", MetaDataFormatUtils.convertToString(statistics.getDateStats().getHighValue()));
+          result.put(COLUMN_MAX, MetaDataFormatUtils.convertToString(statistics.getDateStats().getHighValue()));
         }
         if (statistics.getDateStats().isSetNumNulls()) {
-          result.put("numNulls", statistics.getDateStats().getNumNulls());
+          result.put(COLUMN_NUM_NULLS, statistics.getDateStats().getNumNulls());
         }
         if (statistics.getDateStats().isSetNumDVs()) {
-          result.put("distinctCount", statistics.getDateStats().getNumDVs());
+          result.put(COLUMN_DISTINCT_COUNT, statistics.getDateStats().getNumDVs());
+        }
+      } else if (statistics.isSetTimestampStats()) {
+        if (statistics.getTimestampStats().isSetLowValue()) {
+          result.put(COLUMN_MIN, MetaDataFormatUtils.convertToString(statistics.getTimestampStats().getLowValue()));
+        }
+        if (statistics.getTimestampStats().isSetHighValue()) {
+          result.put(COLUMN_MAX, MetaDataFormatUtils.convertToString(statistics.getTimestampStats().getHighValue()));
+        }
+        if (statistics.getTimestampStats().isSetNumNulls()) {
+          result.put(COLUMN_NUM_NULLS, statistics.getTimestampStats().getNumNulls());
+        }
+        if (statistics.getTimestampStats().isSetNumDVs()) {
+          result.put(COLUMN_DISTINCT_COUNT, statistics.getTimestampStats().getNumDVs());
         }
       }
     }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
index 0586ccb..fef0293 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
@@ -22,6 +22,7 @@ import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.TableType;
@@ -37,6 +38,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.TimestampColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
@@ -56,6 +58,7 @@ import org.apache.hadoop.hive.ql.metadata.UniqueConstraint.UniqueConstraintCol;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo.ForeignKeyCol;
 import org.apache.hadoop.hive.ql.plan.PlanUtils;
 import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hive.common.util.HiveStringUtils;
 
 import com.google.common.collect.Lists;
@@ -112,6 +115,16 @@ public final class MetaDataFormatUtils {
     return writableValue.toString();
   }
 
+  public static String convertToString(org.apache.hadoop.hive.metastore.api.Timestamp val) {
+    if (val == null) {
+      return "";
+    }
+
+    TimestampWritableV2 writableValue = new TimestampWritableV2(
+        Timestamp.ofEpochSecond(val.getSecondsSinceEpoch()));
+    return writableValue.toString();
+  }
+
   private static String convertToString(byte[] buf) {
     if (buf == null || buf.length == 0) {
       return "";
@@ -697,6 +710,13 @@ public final class MetaDataFormatUtils {
                                           "", "",
                                           "", "",
                                           convertToString(dcsd.getBitVectors())));
+        } else if (csd.isSetTimestampStats()) {
+          TimestampColumnStatsData tcsd = csd.getTimestampStats();
+          ret.addAll(Lists.newArrayList(  convertToString(tcsd.getLowValue()), convertToString(tcsd.getHighValue()),
+              "" + tcsd.getNumNulls(), "" + tcsd.getNumDVs(),
+              "", "",
+              "", "",
+              convertToString(tcsd.getBitVectors())));
         }
         // @formatter:on
       } else {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
index feec457..c9284ad 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
@@ -33,6 +33,7 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.Stack;
 
+import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.Context;
@@ -109,6 +110,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFStruct;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -626,6 +628,13 @@ public class StatsRulesProcFactory {
             int minValue = range.minValue.intValue();
             return RangeResult.of(value < minValue, value < maxValue, value == minValue, value == maxValue);
           }
+          case serdeConstants.TIMESTAMP_TYPE_NAME: {
+            TimestampWritableV2 timestampWritable = new TimestampWritableV2(Timestamp.valueOf(boundValue));
+            long value = timestampWritable.getTimestamp().toEpochSecond();
+            long maxValue = range.maxValue.longValue();
+            long minValue = range.minValue.longValue();
+            return RangeResult.of(value < minValue, value < maxValue, value == minValue, value == maxValue);
+          }
           case serdeConstants.BIGINT_TYPE_NAME: {
             long value = Long.parseLong(boundValue);
             long maxValue = range.maxValue.longValue();
@@ -776,8 +785,7 @@ public class StatsRulesProcFactory {
     }
 
     private long evaluateNotExpr(Statistics stats, ExprNodeDesc pred, long currNumRows,
-        AnnotateStatsProcCtx aspCtx, List<String> neededCols, Operator<?> op)
-        throws SemanticException {
+        AnnotateStatsProcCtx aspCtx, List<String> neededCols, Operator<?> op) throws SemanticException {
 
       long numRows = currNumRows;
 
@@ -1045,8 +1053,15 @@ public class StatsRulesProcFactory {
                 return Math.round(((double) (maxValue - value) / (maxValue - minValue)) * numRows);
               }
             }
-          } else if (colTypeLowerCase.equals(serdeConstants.BIGINT_TYPE_NAME)) {
-            long value = Long.parseLong(boundValue);
+          } else if (colTypeLowerCase.equals(serdeConstants.BIGINT_TYPE_NAME) ||
+              colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME)) {
+            long value;
+            if (colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME)) {
+              TimestampWritableV2 timestampWritable = new TimestampWritableV2(Timestamp.valueOf(boundValue));
+              value = timestampWritable.getTimestamp().toEpochSecond();
+            } else {
+              value = Long.parseLong(boundValue);
+            }
             long maxValue = cs.getRange().maxValue.longValue();
             long minValue = cs.getRange().minValue.longValue();
             if (upperBound) {
@@ -1640,6 +1655,7 @@ public class StatsRulesProcFactory {
             case serdeConstants.DATE_TYPE_NAME:
             case serdeConstants.INT_TYPE_NAME:
             case serdeConstants.BIGINT_TYPE_NAME:
+            case serdeConstants.TIMESTAMP_TYPE_NAME:
               long maxValueLong = range.maxValue.longValue();
               long minValueLong = range.minValue.longValue();
               // If min value is less or equal to max value (legal)
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/ColumnStatisticsObjTranslator.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/ColumnStatisticsObjTranslator.java
index 31c9682..e6926d3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/ColumnStatisticsObjTranslator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/ColumnStatisticsObjTranslator.java
@@ -28,14 +28,17 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.Date;
 import org.apache.hadoop.hive.metastore.api.Decimal;
+import org.apache.hadoop.hive.metastore.api.Timestamp;
 import org.apache.hadoop.hive.metastore.api.utils.DecimalUtils;
 import org.apache.hadoop.hive.metastore.columnstats.cache.DateColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.DecimalColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.DoubleColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.LongColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.StringColumnStatsDataInspector;
+import org.apache.hadoop.hive.metastore.columnstats.cache.TimestampColumnStatsDataInspector;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
@@ -46,9 +49,15 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspe
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ColumnStatisticsObjTranslator {
 
+  private static transient final Logger LOG = LoggerFactory
+      .getLogger(ColumnStatisticsObjTranslator.class);
+
   public static ColumnStatisticsObj readHiveStruct(String columnName, String columnType, StructField structField, Object values)
       throws HiveException
   {
@@ -210,6 +219,26 @@ public class ColumnStatisticsObjTranslator {
     }
   }
 
+  private static void unpackTimestampStats(ObjectInspector oi, Object o, String fName, ColumnStatisticsObj statsObj) {
+    if (fName.equals("countnulls")) {
+      long v = ((LongObjectInspector) oi).get(o);
+      statsObj.getStatsData().getTimestampStats().setNumNulls(v);
+    } else if (fName.equals("numdistinctvalues")) {
+      long v = ((LongObjectInspector) oi).get(o);
+      statsObj.getStatsData().getTimestampStats().setNumDVs(v);
+    } else if (fName.equals("max")) {
+      TimestampWritableV2 v = ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o);
+      statsObj.getStatsData().getTimestampStats().setHighValue(new Timestamp(v.getSeconds()));
+    } else if (fName.equals("min")) {
+      TimestampWritableV2 v = ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o);
+      statsObj.getStatsData().getTimestampStats().setLowValue(new Timestamp(v.getSeconds()));
+    } else if (fName.equals("ndvbitvector")) {
+      PrimitiveObjectInspector poi = (PrimitiveObjectInspector) oi;
+      byte[] buf = ((BinaryObjectInspector) poi).getPrimitiveJavaObject(o);
+      statsObj.getStatsData().getTimestampStats().setBitVectors(buf);
+    }
+  }
+
   private static void unpackPrimitiveObject(ObjectInspector oi, Object o, String fieldName, ColumnStatisticsObj statsObj) throws UnsupportedDoubleException {
     if (o == null) {
       return;
@@ -248,6 +277,10 @@ public class ColumnStatisticsObjTranslator {
         DateColumnStatsDataInspector dateStats = new DateColumnStatsDataInspector();
         statsData.setDateStats(dateStats);
         statsObj.setStatsData(statsData);
+      } else if (s.equalsIgnoreCase("timestamp")) {
+        TimestampColumnStatsDataInspector timestampStats = new TimestampColumnStatsDataInspector();
+        statsData.setTimestampStats(timestampStats);
+        statsObj.setStatsData(statsData);
       }
     } else {
       // invoke the right unpack method depending on data type of the column
@@ -265,6 +298,8 @@ public class ColumnStatisticsObjTranslator {
         unpackDecimalStats(oi, o, fieldName, statsObj);
       } else if (statsObj.getStatsData().isSetDateStats()) {
         unpackDateStats(oi, o, fieldName, statsObj);
+      } else if (statsObj.getStatsData().isSetTimestampStats()) {
+        unpackTimestampStats(oi, o, fieldName, statsObj);
       }
     }
   }
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 be52709..0e12f27 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
@@ -131,6 +131,10 @@ public class StatsUtils {
   private static final int DATE_RANGE_LOWER_LIMIT = 10593;
   // Range upper limit for date type when not defined (days, heuristic): '2024-12-31'
   private static final int DATE_RANGE_UPPER_LIMIT = 20089;
+  // Range lower limit for timestamp type when not defined (seconds, heuristic): '1999-01-01 00:00:00'
+  private static final long TIMESTAMP_RANGE_LOWER_LIMIT = 915148800L;
+  // Range upper limit for timestamp type when not defined (seconds, heuristic): '2024-12-31 23:59:59'
+  private static final long TIMESTAMP_RANGE_UPPER_LIMIT = 1735689599L;
 
   /**
    * Collect table, partition and column level statistics
@@ -860,8 +864,15 @@ public class StatsUtils {
     } else if (colTypeLowerCase.equals(serdeConstants.BINARY_TYPE_NAME)) {
       cs.setAvgColLen(csd.getBinaryStats().getAvgColLen());
       cs.setNumNulls(csd.getBinaryStats().getNumNulls());
-    } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME) ||
-        colTypeLowerCase.equals(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME)) {
+    } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME)) {
+      cs.setAvgColLen(JavaDataModel.get().lengthOfTimestamp());
+      cs.setNumNulls(csd.getTimestampStats().getNumNulls());
+      Long lowVal = (csd.getTimestampStats().getLowValue() != null) ? csd.getTimestampStats().getLowValue()
+          .getSecondsSinceEpoch() : null;
+      Long highVal = (csd.getTimestampStats().getHighValue() != null) ? csd.getTimestampStats().getHighValue()
+          .getSecondsSinceEpoch() : null;
+      cs.setRange(lowVal, highVal);
+    } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME)) {
       cs.setAvgColLen(JavaDataModel.get().lengthOfTimestamp());
     } else if (colTypeLowerCase.startsWith(serdeConstants.DECIMAL_TYPE_NAME)) {
       cs.setAvgColLen(JavaDataModel.get().lengthOfDecimal());
@@ -939,8 +950,11 @@ public class StatsUtils {
         cs.setNumTrues(Math.max(1, numRows/2));
         cs.setNumFalses(Math.max(1, numRows/2));
         cs.setAvgColLen(JavaDataModel.get().primitive1());
-    } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME) ||
-        colTypeLowerCase.equals(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME)) {
+    } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME)) {
+      cs.setAvgColLen(JavaDataModel.get().lengthOfTimestamp());
+      // epoch, seconds since epoch
+      cs.setRange(TIMESTAMP_RANGE_LOWER_LIMIT, TIMESTAMP_RANGE_UPPER_LIMIT);
+    } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME)) {
       cs.setAvgColLen(JavaDataModel.get().lengthOfTimestamp());
     } else if (colTypeLowerCase.startsWith(serdeConstants.DECIMAL_TYPE_NAME)) {
       cs.setAvgColLen(JavaDataModel.get().lengthOfDecimal());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
index 642f42b..4e7c598 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.util.JavaDataModel;
 import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
@@ -44,6 +45,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalOb
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableLongObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -83,7 +85,6 @@ public class GenericUDAFComputeStats extends AbstractGenericUDAFResolver {
     case SHORT:
     case INT:
     case LONG:
-    case TIMESTAMP:
     case TIMESTAMPLOCALTZ:
       return new GenericUDAFLongStatsEvaluator();
     case FLOAT:
@@ -99,6 +100,8 @@ public class GenericUDAFComputeStats extends AbstractGenericUDAFResolver {
       return new GenericUDAFDecimalStatsEvaluator();
     case DATE:
       return new GenericUDAFDateStatsEvaluator();
+    case TIMESTAMP:
+      return new GenericUDAFTimestampStatsEvaluator();
     default:
       throw new UDFArgumentTypeException(0,
           "Only integer/long/timestamp/date/float/double/string/binary/boolean/decimal type argument " +
@@ -381,6 +384,8 @@ public class GenericUDAFComputeStats extends AbstractGenericUDAFResolver {
 
         partialResult = new Object[6];
         partialResult[0] = new Text();
+        partialResult[1] = null;
+        partialResult[2] = null;
         partialResult[3] = new LongWritable(0);
         partialResult[4] = new BytesWritable();
 
@@ -405,6 +410,8 @@ public class GenericUDAFComputeStats extends AbstractGenericUDAFResolver {
 
         result = new Object[6];
         result[0] = new Text();
+        result[1] = null;
+        result[2] = null;
         result[3] = new LongWritable(0);
         result[4] = new LongWritable(0);
         result[5] = new BytesWritable();
@@ -1366,6 +1373,80 @@ public class GenericUDAFComputeStats extends AbstractGenericUDAFResolver {
       ((NumericStatsAgg)agg).reset("Date");
     }
   }
+
+  /**
+   * GenericUDAFTimestampStatsEvaluator
+   * High/low value will be saved in stats DB as long value representing seconds since epoch.
+   */
+  public static class GenericUDAFTimestampStatsEvaluator
+      extends GenericUDAFNumericStatsEvaluator<TimestampWritableV2, TimestampObjectInspector> {
+
+    @Override
+    protected TimestampObjectInspector getValueObjectInspector() {
+      return PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
+    }
+
+    @Override
+    protected TimestampObjectInspector getValueObjectInspector(PrimitiveTypeInfo typeInfo) {
+      return getValueObjectInspector();
+    }
+
+    @AggregationType(estimable = true)
+    public class TimestampStatsAgg extends NumericStatsAgg {
+      @Override
+      public int estimate() {
+        JavaDataModel model = JavaDataModel.get();
+        return super.estimate() + model.primitive2() * 2;
+      }
+
+      @Override
+      protected void update(Object p, PrimitiveObjectInspector inputOI) {
+        // TimestampWritableV2 is mutable, TimestampStatsAgg needs its own copy
+        TimestampWritableV2 v = new TimestampWritableV2((TimestampWritableV2) inputOI.getPrimitiveWritableObject(p));
+
+        //Update min counter if new value is less than min seen so far
+        if (min == null || v.compareTo(min) < 0) {
+          min = v;
+        }
+        //Update max counter if new value is greater than max seen so far
+        if (max == null || v.compareTo(max) > 0) {
+          max = v;
+        }
+        // Add value to NumDistinctValue Estimator
+        numDV.addToEstimator(v.getSeconds());
+      }
+
+      @Override
+      protected void updateMin(Object minValue, TimestampObjectInspector minFieldOI) {
+        if ((minValue != null) && (min == null ||
+            min.compareTo(minFieldOI.getPrimitiveWritableObject(minValue)) > 0)) {
+          // TimestampWritableV2 is mutable, TimestampStatsAgg needs its own copy
+          min = new TimestampWritableV2(minFieldOI.getPrimitiveWritableObject(minValue));
+        }
+      }
+
+      @Override
+      protected void updateMax(Object maxValue, TimestampObjectInspector maxFieldOI) {
+        if ((maxValue != null) && (max == null ||
+            max.compareTo(maxFieldOI.getPrimitiveWritableObject(maxValue)) < 0)) {
+          // TimestampWritableV2 is mutable, TimestampStatsAgg needs its own copy
+          max = new TimestampWritableV2(maxFieldOI.getPrimitiveWritableObject(maxValue));
+        }
+      }
+    };
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new TimestampStatsAgg();
+      reset(result);
+      return result;
+    }
+
+    @Override
+    public void reset(AggregationBuffer agg) throws HiveException {
+      ((NumericStatsAgg)agg).reset("Timestamp");
+    }
+  }
   
   @InterfaceAudience.LimitedPrivate(value = { "Hive" })
   static int lengthFor(JavaDataModel model, Integer numVector) {
diff --git a/ql/src/test/queries/clientpositive/timestamp_comparison3.q b/ql/src/test/queries/clientpositive/timestamp_comparison3.q
new file mode 100644
index 0000000..145d89b
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/timestamp_comparison3.q
@@ -0,0 +1,32 @@
+set hive.fetch.task.conversion=none;
+set hive.stats.fetch.column.stats=true;
+
+create database timestamp_test_n123;
+create table timestamp_test_n123.onecolumntable (ts timestamp);
+
+insert into timestamp_test_n123.onecolumntable values
+('2015-01-01 00:00:00'),
+('2015-01-02 00:00:00'),
+('2015-01-03 00:00:00'),
+('2015-01-04 00:00:00'),
+('2015-01-05 00:00:00');
+
+describe formatted timestamp_test_n123.onecolumntable ts;
+
+explain
+select ts from timestamp_test_n123.onecolumntable
+where ts >= cast('2015-01-02 00:00:00' as timestamp)
+  and ts <= cast('2015-01-04 00:00:00' as timestamp);
+
+explain
+select ts from timestamp_test_n123.onecolumntable
+where ts >= cast('2015-01-02 00:00:00' as timestamp)
+  and ts <= cast('2015-01-03 00:00:00' as timestamp);
+
+explain
+select ts from timestamp_test_n123.onecolumntable
+where ts >= cast('2015-01-01 00:00:00' as timestamp)
+  and ts <= cast('2015-01-08 00:00:00' as timestamp);
+
+drop table timestamp_test_n123.onecolumntable;
+drop database timestamp_test_n123;
diff --git a/ql/src/test/results/clientpositive/alter_table_update_status.q.out b/ql/src/test/results/clientpositive/alter_table_update_status.q.out
index 9ffd9b4..6747c3b 100644
--- a/ql/src/test/results/clientpositive/alter_table_update_status.q.out
+++ b/ql/src/test/results/clientpositive/alter_table_update_status.q.out
@@ -453,8 +453,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats_n0
 col_name            	ts                  
 data_type           	timestamp           
-min                 	1325379723          
-max                 	1325379723          
+min                 	2012-01-01 01:02:03 
+max                 	2012-01-01 01:02:03 
 num_nulls           	1                   
 distinct_count      	1                   
 avg_col_len         	                    
@@ -908,8 +908,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats_n0
 col_name            	ts                  
 data_type           	timestamp           
-min                 	1325379723          
-max                 	1325379723          
+min                 	2012-01-01 01:02:03 
+max                 	2012-01-01 01:02:03 
 num_nulls           	1                   
 distinct_count      	1                   
 avg_col_len         	                    
@@ -935,8 +935,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats_n0
 col_name            	ts                  
 data_type           	timestamp           
-min                 	1357030924          
-max                 	1357030923          
+min                 	2013-01-01 09:02:04 
+max                 	2013-01-01 09:02:03 
 num_nulls           	12                  
 distinct_count      	7                   
 avg_col_len         	                    
diff --git a/ql/src/test/results/clientpositive/alter_table_update_status_disable_bitvector.q.out b/ql/src/test/results/clientpositive/alter_table_update_status_disable_bitvector.q.out
index ec6a780..4fb4a70d 100644
--- a/ql/src/test/results/clientpositive/alter_table_update_status_disable_bitvector.q.out
+++ b/ql/src/test/results/clientpositive/alter_table_update_status_disable_bitvector.q.out
@@ -453,8 +453,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats
 col_name            	ts                  
 data_type           	timestamp           
-min                 	1325379723          
-max                 	1325379723          
+min                 	2012-01-01 01:02:03 
+max                 	2012-01-01 01:02:03 
 num_nulls           	1                   
 distinct_count      	1                   
 avg_col_len         	                    
@@ -908,8 +908,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats
 col_name            	ts                  
 data_type           	timestamp           
-min                 	1325379723          
-max                 	1325379723          
+min                 	2012-01-01 01:02:03 
+max                 	2012-01-01 01:02:03 
 num_nulls           	1                   
 distinct_count      	1                   
 avg_col_len         	                    
@@ -935,8 +935,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats
 col_name            	ts                  
 data_type           	timestamp           
-min                 	1357030924          
-max                 	1357030923          
+min                 	2013-01-01 09:02:04 
+max                 	2013-01-01 09:02:03 
 num_nulls           	12                  
 distinct_count      	7                   
 avg_col_len         	                    
diff --git a/ql/src/test/results/clientpositive/beeline/desc_table_formatted.q.out b/ql/src/test/results/clientpositive/beeline/desc_table_formatted.q.out
index e961744..2f7478d 100644
--- a/ql/src/test/results/clientpositive/beeline/desc_table_formatted.q.out
+++ b/ql/src/test/results/clientpositive/beeline/desc_table_formatted.q.out
@@ -222,8 +222,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats_n0
 col_name	ts
 data_type	timestamp
-min	1325379722
-max	1325379723
+min	2012-01-01 01:02:02
+max	2012-01-01 01:02:03
 num_nulls	1
 distinct_count	2
 avg_col_len	
@@ -585,7 +585,7 @@ PREHOOK: Input: default@datatype_stats_n0
 POSTHOOK: query: DESC FORMATTED datatype_stats_n0 ts
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats_n0
-{"columns":[{"name":"ts","type":"timestamp","comment":"from deserializer","min":1325379722,"max":1325379723,"numNulls":1,"distinctCount":2}]}
+{"columns":[{"name":"ts","type":"timestamp","comment":"from deserializer","min":"2012-01-01 01:02:02","max":"2012-01-01 01:02:03","numNulls":1,"distinctCount":2}]}
 PREHOOK: query: DESC FORMATTED datatype_stats_n0 dt
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@datatype_stats_n0
diff --git a/ql/src/test/results/clientpositive/confirm_initial_tbl_stats.q.out b/ql/src/test/results/clientpositive/confirm_initial_tbl_stats.q.out
index f259a1e..cdd934c 100644
--- a/ql/src/test/results/clientpositive/confirm_initial_tbl_stats.q.out
+++ b/ql/src/test/results/clientpositive/confirm_initial_tbl_stats.q.out
@@ -272,8 +272,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@alltypesorc
 col_name            	ctimestamp1         
 data_type           	timestamp           
-min                 	-28830              
-max                 	-28769              
+min                 	1969-12-31 15:59:30 
+max                 	1969-12-31 16:00:31 
 num_nulls           	3115                
 distinct_count      	35                  
 avg_col_len         	                    
diff --git a/ql/src/test/results/clientpositive/constprog_type.q.out b/ql/src/test/results/clientpositive/constprog_type.q.out
index 983d5ba..159abf9 100644
--- a/ql/src/test/results/clientpositive/constprog_type.q.out
+++ b/ql/src/test/results/clientpositive/constprog_type.q.out
@@ -59,20 +59,20 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 944 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 1008 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     sort order: 
-                    Statistics: Num rows: 1 Data size: 944 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: struct<columntype:string,min:date,max:date,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>)
+                    Statistics: Num rows: 1 Data size: 1008 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: struct<columntype:string,min:date,max:date,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,min:timestamp,max:timestamp,countnulls:bigint,bitvector:binary>)
       Reduce Operator Tree:
         Group By Operator
           aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 1 Data size: 976 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 1040 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 976 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 1040 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/desc_table_formatted.q.out b/ql/src/test/results/clientpositive/desc_table_formatted.q.out
index 0a5c363..9014844 100644
--- a/ql/src/test/results/clientpositive/desc_table_formatted.q.out
+++ b/ql/src/test/results/clientpositive/desc_table_formatted.q.out
@@ -222,8 +222,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats_n0
 col_name            	ts                  
 data_type           	timestamp           
-min                 	1325379722          
-max                 	1325379723          
+min                 	2012-01-01 01:02:02 
+max                 	2012-01-01 01:02:03 
 num_nulls           	1                   
 distinct_count      	2                   
 avg_col_len         	                    
@@ -585,7 +585,7 @@ PREHOOK: Input: default@datatype_stats_n0
 POSTHOOK: query: DESC FORMATTED datatype_stats_n0 ts
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats_n0
-{"columns":[{"name":"ts","type":"timestamp","comment":"from deserializer","min":1325379722,"max":1325379723,"numNulls":1,"distinctCount":2}]}
+{"columns":[{"name":"ts","type":"timestamp","comment":"from deserializer","min":"2012-01-01 01:02:02","max":"2012-01-01 01:02:03","numNulls":1,"distinctCount":2}]}
 PREHOOK: query: DESC FORMATTED datatype_stats_n0 dt
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@datatype_stats_n0
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_dynamic_partition.q.out b/ql/src/test/results/clientpositive/druid/druidmini_dynamic_partition.q.out
index 09636a4..afc876b 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_dynamic_partition.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_dynamic_partition.q.out
@@ -125,23 +125,23 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ctimestamp1 is not null (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2477090 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ctimestamp1 is not null (type: boolean)
-                    Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9173 Data size: 1849230 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: CAST( ctimestamp1 AS timestamp with local time zone) (type: timestamp with local time zone), cstring1 (type: string), cstring2 (type: string), cdouble (type: double), cfloat (type: float), ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cboolean1 (type: boolean), cboolean2 (type: boolean)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-                      Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9173 Data size: 1942190 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col0 (type: timestamp with local time zone), _col1 (type: string), _col2 (type: string), _col3 (type: double), _col4 (type: float), _col5 (type: tinyint), _col6 (type: smallint), _col7 (type: int), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: boolean), floor_hour(CAST( GenericUDFEpochMilli(_col0) AS TIMESTAMP)) (type: timestamp), (floor((1.0D / rand())) % 6) (type: bigint)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, __time_granularity, __druid_extra_partition_key
-                        Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9173 Data size: 1942190 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: __time_granularity (type: timestamp), __druid_extra_partition_key (type: bigint)
                           sort order: ++
                           Map-reduce partition columns: __time_granularity (type: timestamp), __druid_extra_partition_key (type: bigint)
-                          Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9173 Data size: 1942190 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: timestamp with local time zone), _col1 (type: string), _col2 (type: string), _col3 (type: double), _col4 (type: float), _col5 (type: tinyint), _col6 (type: smallint), _col7 (type: int), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: boolean)
             Execution mode: llap
             LLAP IO: all inputs
@@ -151,11 +151,11 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: timestamp with local time zone), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: double), VALUE._col4 (type: float), VALUE._col5 (type: tinyint), VALUE._col6 (type: smallint), VALUE._col7 (type: int), VALUE._col8 (type: bigint), VALUE._col9 (type: boolean), VALUE._col10 (type: boolean), KEY.__time_granularity (type: timestamp), KEY.__druid_extra_partition_key (type: bigint)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, __time_granularity, __druid_extra_partition_key
-                Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9173 Data size: 1942190 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9173 Data size: 1942190 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat
                       output format: org.apache.hadoop.hive.druid.io.DruidOutputFormat
@@ -364,23 +364,23 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ctimestamp2 is not null (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2477130 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ctimestamp2 is not null (type: boolean)
-                    Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9174 Data size: 1849380 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: CAST( ctimestamp2 AS timestamp with local time zone) (type: timestamp with local time zone), cstring1 (type: string), cstring2 (type: string), cdouble (type: double), cfloat (type: float), ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cboolean1 (type: boolean), cboolean2 (type: boolean)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-                      Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9174 Data size: 1942340 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col0 (type: timestamp with local time zone), _col1 (type: string), _col2 (type: string), _col3 (type: double), _col4 (type: float), _col5 (type: tinyint), _col6 (type: smallint), _col7 (type: int), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: boolean), floor_hour(CAST( GenericUDFEpochMilli(_col0) AS TIMESTAMP)) (type: timestamp), (floor((1.0D / rand())) % 6) (type: bigint)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, __time_granularity, __druid_extra_partition_key
-                        Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9174 Data size: 1942340 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: __time_granularity (type: timestamp), __druid_extra_partition_key (type: bigint)
                           sort order: ++
                           Map-reduce partition columns: __time_granularity (type: timestamp), __druid_extra_partition_key (type: bigint)
-                          Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9174 Data size: 1942340 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: timestamp with local time zone), _col1 (type: string), _col2 (type: string), _col3 (type: double), _col4 (type: float), _col5 (type: tinyint), _col6 (type: smallint), _col7 (type: int), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: boolean)
             Execution mode: llap
             LLAP IO: all inputs
@@ -390,11 +390,11 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: timestamp with local time zone), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: double), VALUE._col4 (type: float), VALUE._col5 (type: tinyint), VALUE._col6 (type: smallint), VALUE._col7 (type: int), VALUE._col8 (type: bigint), VALUE._col9 (type: boolean), VALUE._col10 (type: boolean), KEY.__time_granularity (type: timestamp), KEY.__druid_extra_partition_key (type: bigint)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, __time_granularity, __druid_extra_partition_key
-                Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9174 Data size: 1942340 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9174 Data size: 1942340 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat
                       output format: org.apache.hadoop.hive.druid.io.DruidOutputFormat
@@ -505,23 +505,23 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ctimestamp1 is not null (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2477090 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ctimestamp1 is not null (type: boolean)
-                    Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9173 Data size: 1849230 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: CAST( ctimestamp1 AS timestamp with local time zone) (type: timestamp with local time zone), cstring1 (type: string), cstring2 (type: string), cdouble (type: double), cfloat (type: float), ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cboolean1 (type: boolean), cboolean2 (type: boolean)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-                      Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9173 Data size: 1942190 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col0 (type: timestamp with local time zone), _col1 (type: string), _col2 (type: string), _col3 (type: double), _col4 (type: float), _col5 (type: tinyint), _col6 (type: smallint), _col7 (type: int), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: boolean), floor_hour(CAST( GenericUDFEpochMilli(_col0) AS TIMESTAMP)) (type: timestamp), (floor((1.0D / rand())) % 6) (type: bigint)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, __time_granularity, __druid_extra_partition_key
-                        Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9173 Data size: 1942190 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: __time_granularity (type: timestamp), __druid_extra_partition_key (type: bigint)
                           sort order: ++
                           Map-reduce partition columns: __time_granularity (type: timestamp), __druid_extra_partition_key (type: bigint)
-                          Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9173 Data size: 1942190 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: timestamp with local time zone), _col1 (type: string), _col2 (type: string), _col3 (type: double), _col4 (type: float), _col5 (type: tinyint), _col6 (type: smallint), _col7 (type: int), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: boolean)
             Execution mode: llap
             LLAP IO: all inputs
@@ -531,11 +531,11 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: timestamp with local time zone), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: double), VALUE._col4 (type: float), VALUE._col5 (type: tinyint), VALUE._col6 (type: smallint), VALUE._col7 (type: int), VALUE._col8 (type: bigint), VALUE._col9 (type: boolean), VALUE._col10 (type: boolean), KEY.__time_granularity (type: timestamp), KEY.__druid_extra_partition_key (type: bigint)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, __time_granularity, __druid_extra_partition_key
-                Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9173 Data size: 1942190 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9173 Data size: 1942190 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat
                       output format: org.apache.hadoop.hive.druid.io.DruidOutputFormat
diff --git a/ql/src/test/results/clientpositive/foldts.q.out b/ql/src/test/results/clientpositive/foldts.q.out
index d759de8..feda88c 100644
--- a/ql/src/test/results/clientpositive/foldts.q.out
+++ b/ql/src/test/results/clientpositive/foldts.q.out
@@ -18,11 +18,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc
-            Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 366960 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: ctimestamp1 (type: timestamp), to_unix_timestamp(ctimestamp1) (type: bigint), to_unix_timestamp(ctimestamp1) (type: bigint)
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 12288 Data size: 688128 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 563568 Basic stats: COMPLETE Column stats: COMPLETE
               Limit
                 Number of rows: 1
                 Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
@@ -80,7 +80,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc
-            Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 366960 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: from_unixtime(to_unix_timestamp(ctimestamp1), 'EEEE') (type: string)
               outputColumnNames: _col0
@@ -132,7 +132,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc
-            Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 366960 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: from_unixtime(to_unix_timestamp(ctimestamp1), 'EEEE') (type: string)
               outputColumnNames: _col0
diff --git a/ql/src/test/results/clientpositive/interval_arithmetic.q.out b/ql/src/test/results/clientpositive/interval_arithmetic.q.out
index 9d9aef4..3b5db9b 100644
--- a/ql/src/test/results/clientpositive/interval_arithmetic.q.out
+++ b/ql/src/test/results/clientpositive/interval_arithmetic.q.out
@@ -222,11 +222,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: interval_arithmetic_1_n0
-            Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 366960 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: tsval (type: timestamp), (tsval - INTERVAL'2-2') (type: timestamp), (tsval - INTERVAL'-2-2') (type: timestamp), (tsval + INTERVAL'2-2') (type: timestamp), (tsval + INTERVAL'-2-2') (type: timestamp), (INTERVAL'-2-2' + tsval) (type: timestamp), (INTERVAL'2-2' + tsval) (type: timestamp)
               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-              Statistics: Num rows: 12288 Data size: 3440640 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 3316080 Basic stats: COMPLETE Column stats: COMPLETE
               Limit
                 Number of rows: 2
                 Statistics: Num rows: 2 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
@@ -452,11 +452,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: interval_arithmetic_1_n0
-            Statistics: Num rows: 12288 Data size: 1005264 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 880704 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: dateval (type: date), tsval (type: timestamp), (dateval - tsval) (type: interval_day_time), (tsval - dateval) (type: interval_day_time), (tsval - tsval) (type: interval_day_time)
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
-              Statistics: Num rows: 12288 Data size: 1447632 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 1323072 Basic stats: COMPLETE Column stats: COMPLETE
               Limit
                 Number of rows: 2
                 Statistics: Num rows: 2 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
@@ -537,11 +537,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: interval_arithmetic_1_n0
-            Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 366960 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: tsval (type: timestamp), (tsval - INTERVAL'99 11:22:33.123456789') (type: timestamp), (tsval - INTERVAL'-99 11:22:33.123456789') (type: timestamp), (tsval + INTERVAL'99 11:22:33.123456789') (type: timestamp), (tsval + INTERVAL'-99 11:22:33.123456789') (type: timestamp), (INTERVAL'-99 11:22:33.123456789' + tsval) (type: timestamp), (INTERVAL'99 11:22:33.123456789' + tsval) (type: timestamp)
               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-              Statistics: Num rows: 12288 Data size: 3440640 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 3316080 Basic stats: COMPLETE Column stats: COMPLETE
               Limit
                 Number of rows: 2
                 Statistics: Num rows: 2 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_4.q.out b/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
index e919266..258e5f3 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
@@ -30,29 +30,29 @@ Stage-0
     Stage-1
       Reducer 3 vectorized, llap
       File Output Operator [FS_35]
-        Select Operator [SEL_34] (rows=4626 width=552)
+        Select Operator [SEL_34] (rows=4626 width=528)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"]
         <-Reducer 2 [SIMPLE_EDGE] llap
           SHUFFLE [RS_10]
-            Merge Join Operator [MERGEJOIN_27] (rows=4626 width=552)
+            Merge Join Operator [MERGEJOIN_27] (rows=4626 width=528)
               Conds:RS_30._col2=RS_33._col2(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"]
             <-Map 1 [SIMPLE_EDGE] vectorized, llap
               SHUFFLE [RS_30]
                 PartitionCols:_col2
-                Select Operator [SEL_29] (rows=3078 width=251)
+                Select Operator [SEL_29] (rows=3078 width=231)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
-                  Filter Operator [FIL_28] (rows=3078 width=251)
+                  Filter Operator [FIL_28] (rows=3078 width=231)
                     predicate:cint BETWEEN 1000000 AND 3000000
-                    TableScan [TS_0] (rows=12288 width=251)
+                    TableScan [TS_0] (rows=12288 width=231)
                       default@alltypesorc,a,Tbl:COMPLETE,Col:COMPLETE,Output:["ctinyint","csmallint","cint","cbigint","cfloat","cdouble","cstring1","cstring2","ctimestamp1","ctimestamp2","cboolean1","cboolean2"]
             <-Map 4 [SIMPLE_EDGE] vectorized, llap
               SHUFFLE [RS_33]
                 PartitionCols:_col2
-                Select Operator [SEL_32] (rows=2298 width=251)
+                Select Operator [SEL_32] (rows=2298 width=231)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
-                  Filter Operator [FIL_31] (rows=2298 width=251)
+                  Filter Operator [FIL_31] (rows=2298 width=231)
                     predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
-                    TableScan [TS_3] (rows=12288 width=251)
+                    TableScan [TS_3] (rows=12288 width=231)
                       default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["ctinyint","csmallint","cint","cbigint","cfloat","cdouble","cstring1","cstring2","ctimestamp1","ctimestamp2","cboolean1","cboolean2"]
 
 PREHOOK: query: select
diff --git a/ql/src/test/results/clientpositive/llap/orc_llap.q.out b/ql/src/test/results/clientpositive/llap/orc_llap.q.out
index 0ad9682..6a0d855 100644
--- a/ql/src/test/results/clientpositive/llap/orc_llap.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_llap.q.out
@@ -344,14 +344,14 @@ STAGE PLANS:
                 TableScan
                   alias: orc_llap
                   filterExpr: ((cint > 10) and cbigint is not null) (type: boolean)
-                  Statistics: Num rows: 122880 Data size: 30929630 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 122880 Data size: 28438110 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((cint > 10) and cbigint is not null) (type: boolean)
-                    Statistics: Num rows: 45873 Data size: 11546630 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 45873 Data size: 10616550 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: hash(ctinyint,csmallint,cint,cbigint,cfloat,cdouble,cstring1,cstring2,ctimestamp1,ctimestamp2,cboolean1,cboolean2) (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 45873 Data size: 11546630 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 45873 Data size: 10616550 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0)
                         minReductionHashAggr: 0.99
@@ -828,14 +828,14 @@ STAGE PLANS:
                 TableScan
                   alias: orc_llap
                   filterExpr: ((cint > 10) and cbigint is not null) (type: boolean)
-                  Statistics: Num rows: 245760 Data size: 61859030 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 245760 Data size: 56875910 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((cint > 10) and cbigint is not null) (type: boolean)
-                    Statistics: Num rows: 91747 Data size: 23093340 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 91747 Data size: 21233100 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: hash(ctinyint,csmallint,cint,cbigint,cfloat,cdouble,cstring1,cstring2,ctimestamp1,ctimestamp2,cboolean1,cboolean2) (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 91747 Data size: 23093340 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 91747 Data size: 21233100 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0)
                         minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/llap/orc_llap_nonvector.q.out b/ql/src/test/results/clientpositive/llap/orc_llap_nonvector.q.out
index 472b310..d656c3c 100644
--- a/ql/src/test/results/clientpositive/llap/orc_llap_nonvector.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_llap_nonvector.q.out
@@ -88,17 +88,17 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: orc_llap_nonvector
-                  Statistics: Num rows: 12288 Data size: 3191474 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2942394 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean), rdm (type: double)
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                    Statistics: Num rows: 12288 Data size: 3191474 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12288 Data size: 2942394 Basic stats: COMPLETE Column stats: COMPLETE
                     Limit
                       Number of rows: 100
-                      Statistics: Num rows: 100 Data size: 26280 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 100 Data size: 24360 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 100 Data size: 26280 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 100 Data size: 24360 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/orc_merge5.q.out b/ql/src/test/results/clientpositive/llap/orc_merge5.q.out
index 6790cd7..4924b8a 100644
--- a/ql/src/test/results/clientpositive/llap/orc_merge5.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_merge5.q.out
@@ -74,11 +74,11 @@ STAGE PLANS:
                           minReductionHashAggr: 0.99
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                          Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
                             sort order: 
-                            Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
-                            value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: s [...]
+                            Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                            value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: s [...]
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -88,10 +88,10 @@ STAGE PLANS:
                 aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2), compute_stats(VALUE._col3), compute_stats(VALUE._col4)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -209,11 +209,11 @@ STAGE PLANS:
                           minReductionHashAggr: 0.99
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                          Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
                             sort order: 
-                            Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
-                            value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: s [...]
+                            Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                            value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: s [...]
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -223,10 +223,10 @@ STAGE PLANS:
                 aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2), compute_stats(VALUE._col3), compute_stats(VALUE._col4)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
                   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/orc_merge6.q.out b/ql/src/test/results/clientpositive/llap/orc_merge6.q.out
index 7021220..9cab015 100644
--- a/ql/src/test/results/clientpositive/llap/orc_merge6.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_merge6.q.out
@@ -81,7 +81,7 @@ STAGE PLANS:
                             sort order: ++
                             Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
                             Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
-                            value expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col6 (type: s [...]
+                            value expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col6 (type: s [...]
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -94,7 +94,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38, [...]
+                  expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38, [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
@@ -269,7 +269,7 @@ STAGE PLANS:
                             sort order: ++
                             Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
                             Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
-                            value expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col6 (type: s [...]
+                            value expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col6 (type: s [...]
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -282,7 +282,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38, [...]
+                  expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38, [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
diff --git a/ql/src/test/results/clientpositive/llap/orc_merge7.q.out b/ql/src/test/results/clientpositive/llap/orc_merge7.q.out
index 16bcc5a..f60bf9d 100644
--- a/ql/src/test/results/clientpositive/llap/orc_merge7.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_merge7.q.out
@@ -69,7 +69,7 @@ STAGE PLANS:
                           sort order: +
                           Map-reduce partition columns: _col0 (type: double)
                           Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col5 (type: str [...]
+                          value expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col5 (type: str [...]
                     Reduce Output Operator
                       key expressions: _col5 (type: double)
                       sort order: +
@@ -88,7 +88,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                 Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38, [...]
+                  expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38, [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                   Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
@@ -304,7 +304,7 @@ STAGE PLANS:
                           sort order: +
                           Map-reduce partition columns: _col0 (type: double)
                           Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col5 (type: str [...]
+                          value expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col5 (type: str [...]
                     Reduce Output Operator
                       key expressions: _col5 (type: double)
                       sort order: +
@@ -323,7 +323,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                 Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38, [...]
+                  expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38, [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                   Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
diff --git a/ql/src/test/results/clientpositive/llap/orc_merge_incompat1.q.out b/ql/src/test/results/clientpositive/llap/orc_merge_incompat1.q.out
index b299d1e..d918e18 100644
--- a/ql/src/test/results/clientpositive/llap/orc_merge_incompat1.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_merge_incompat1.q.out
@@ -74,11 +74,11 @@ STAGE PLANS:
                           minReductionHashAggr: 0.99
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                          Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
                             sort order: 
-                            Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
-                            value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: s [...]
+                            Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                            value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: s [...]
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -88,10 +88,10 @@ STAGE PLANS:
                 aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2), compute_stats(VALUE._col3), compute_stats(VALUE._col4)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
                   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/orc_merge_incompat2.q.out b/ql/src/test/results/clientpositive/llap/orc_merge_incompat2.q.out
index 2d2787d..d970134 100644
--- a/ql/src/test/results/clientpositive/llap/orc_merge_incompat2.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_merge_incompat2.q.out
@@ -69,7 +69,7 @@ STAGE PLANS:
                           sort order: +
                           Map-reduce partition columns: _col0 (type: double)
                           Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col5 (type: str [...]
+                          value expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col5 (type: str [...]
                     Reduce Output Operator
                       key expressions: _col5 (type: double)
                       sort order: +
@@ -88,7 +88,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                 Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38, [...]
+                  expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38, [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                   Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
diff --git a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_2.q.out b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_2.q.out
index 7b1de57..c339430 100644
--- a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_2.q.out
@@ -47,19 +47,19 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 4615 Data size: 1161780 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1068260 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean), UDFToInteger(csmallint) (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                      Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col12 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col12 (type: int)
-                        Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -102,12 +102,12 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col12 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col12 (type: int)
-                  Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 3 
             Execution mode: llap
@@ -119,11 +119,11 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
                   sort order: +++
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -131,10 +131,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -252,19 +252,19 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 4615 Data size: 1161780 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1068260 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean), UDFToInteger(csmallint) (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                      Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col12 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col12 (type: int)
-                        Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -307,12 +307,12 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col12 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col12 (type: int)
-                  Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 3 
             Execution mode: llap
@@ -324,11 +324,11 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
                   sort order: +++
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -336,10 +336,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -457,19 +457,19 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 4615 Data size: 1161780 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1068260 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean), UDFToInteger(csmallint) (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                      Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col12 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col12 (type: int)
-                        Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -512,12 +512,12 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col12 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col12 (type: int)
-                  Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 3 
             Execution mode: llap
@@ -529,11 +529,11 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
                   sort order: +++
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -541,10 +541,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 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/tez_dynpart_hashjoin_3.q.out b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_3.q.out
index 4866a42..4fa69f6 100644
--- a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_3.q.out
@@ -33,11 +33,11 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean), (cint < 100) (type: boolean)
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                    Statistics: Num rows: 12288 Data size: 3142322 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12288 Data size: 2893242 Basic stats: COMPLETE Column stats: COMPLETE
                     Limit
                       Number of rows: 1
                       Statistics: Num rows: 1 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
@@ -149,11 +149,11 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean), (cint < 100) (type: boolean)
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                    Statistics: Num rows: 12288 Data size: 3142322 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12288 Data size: 2893242 Basic stats: COMPLETE Column stats: COMPLETE
                     Limit
                       Number of rows: 1
                       Statistics: Num rows: 1 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
@@ -258,11 +258,11 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                    Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                     Limit
                       Number of rows: 1
                       Statistics: Num rows: 1 Data size: 310 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_2.q.out b/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_2.q.out
index 7b1de57..c339430 100644
--- a/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_2.q.out
@@ -47,19 +47,19 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 4615 Data size: 1161780 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1068260 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean), UDFToInteger(csmallint) (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                      Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col12 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col12 (type: int)
-                        Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -102,12 +102,12 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col12 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col12 (type: int)
-                  Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 3 
             Execution mode: llap
@@ -119,11 +119,11 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
                   sort order: +++
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -131,10 +131,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -252,19 +252,19 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 4615 Data size: 1161780 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1068260 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean), UDFToInteger(csmallint) (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                      Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col12 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col12 (type: int)
-                        Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -307,12 +307,12 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col12 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col12 (type: int)
-                  Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 3 
             Execution mode: llap
@@ -324,11 +324,11 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
                   sort order: +++
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -336,10 +336,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -457,19 +457,19 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 4615 Data size: 1161780 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1068260 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean), UDFToInteger(csmallint) (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                      Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col12 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col12 (type: int)
-                        Statistics: Num rows: 4615 Data size: 1175564 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1082044 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -512,12 +512,12 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col12 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col12 (type: int)
-                  Statistics: Num rows: 1084 Data size: 86954 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 314 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 3 
             Execution mode: llap
@@ -529,11 +529,11 @@ STAGE PLANS:
                   0 _col12 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
                   sort order: +++
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -541,10 +541,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 195920 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 195920 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_aggregate_9.q.out b/ql/src/test/results/clientpositive/llap/vector_aggregate_9.q.out
index 7ba6b4b..ef516b5 100644
--- a/ql/src/test/results/clientpositive/llap/vector_aggregate_9.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_aggregate_9.q.out
@@ -419,7 +419,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: vectortab2korc_n4
-                  Statistics: Num rows: 2000 Data size: 80000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2000 Data size: 75760 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:t:tinyint, 1:si:smallint, 2:i:int, 3:b:bigint, 4:f:float, 5:d:double, 6:dc:decimal(38,18), 7:bo:boolean, 8:s:string, 9:s2:string, 10:ts:timestamp, 11:ts2:timestamp, 12:dt:date, 13:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -430,7 +430,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [10]
-                    Statistics: Num rows: 2000 Data size: 80000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2000 Data size: 75760 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: min(ts), max(ts), sum(ts), count(ts)
                       Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out b/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
index 3d70e15..9573b3d 100644
--- a/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
@@ -347,7 +347,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprOrExpr(children: SelectColumnIsNotNull(col 8:timestamp), SelectColumnIsNotNull(col 9:timestamp))
+                        predicateExpression: FilterExprOrExpr(children: SelectColumnIsNotNull(col 9:timestamp), SelectColumnIsNotNull(col 8:timestamp))
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out b/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out
index c30bd07..bd6cd8c 100644
--- a/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out
@@ -51,7 +51,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: cint is not null (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -60,7 +60,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: SelectColumnIsNotNull(col 2:int)
                     predicate: cint is not null (type: boolean)
-                    Statistics: Num rows: 9173 Data size: 2309110 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9173 Data size: 2123190 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
@@ -68,7 +68,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]
-                      Statistics: Num rows: 9173 Data size: 2309110 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9173 Data size: 2123190 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -83,13 +83,13 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 10090 Data size: 2540021 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 10090 Data size: 2335509 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
                           compressed: false
                           File Sink Vectorization:
                               className: VectorFileSinkOperator
                               native: false
-                          Statistics: Num rows: 10090 Data size: 2540021 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 10090 Data size: 2335509 Basic stats: COMPLETE Column stats: NONE
                           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_decimal_cast.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_cast.q.out
index f558cfc..6153137 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_cast.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_cast.q.out
@@ -23,8 +23,8 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc
-                  filterExpr: (cboolean1 is not null and cint is not null and cdouble is not null and ctimestamp1 is not null) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 638316 Basic stats: COMPLETE Column stats: COMPLETE
+                  filterExpr: (cboolean1 is not null and cint is not null and ctimestamp1 is not null and cdouble is not null) (type: boolean)
+                  Statistics: Num rows: 12288 Data size: 513756 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -32,9 +32,9 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 10:boolean), SelectColumnIsNotNull(col 2:int), SelectColumnIsNotNull(col 5:double), SelectColumnIsNotNull(col 8:timestamp))
-                    predicate: (cboolean1 is not null and cint is not null and cdouble is not null and ctimestamp1 is not null) (type: boolean)
-                    Statistics: Num rows: 5112 Data size: 265564 Basic stats: COMPLETE Column stats: COMPLETE
+                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 10:boolean), SelectColumnIsNotNull(col 2:int), SelectColumnIsNotNull(col 8:timestamp), SelectColumnIsNotNull(col 5:double))
+                    predicate: (cboolean1 is not null and cint is not null and ctimestamp1 is not null and cdouble is not null) (type: boolean)
+                    Statistics: Num rows: 3816 Data size: 159600 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cdouble (type: double), cint (type: int), cboolean1 (type: boolean), ctimestamp1 (type: timestamp), CAST( cdouble AS decimal(20,10)) (type: decimal(20,10)), CAST( cint AS decimal(23,14)) (type: decimal(23,14)), CAST( cboolean1 AS decimal(5,2)) (type: decimal(5,2)), CAST( ctimestamp1 AS decimal(15,0)) (type: decimal(15,0))
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
@@ -43,19 +43,19 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [5, 2, 10, 8, 13, 14, 15, 16]
                           selectExpressions: CastDoubleToDecimal(col 5:double) -> 13:decimal(20,10), CastLongToDecimal(col 2:int) -> 14:decimal(23,14), CastLongToDecimal(col 10:boolean) -> 15:decimal(5,2), CastTimestampToDecimal(col 8:timestamp) -> 16:decimal(15,0)
-                      Statistics: Num rows: 5112 Data size: 2410700 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3816 Data size: 1760976 Basic stats: COMPLETE Column stats: COMPLETE
                       Limit
                         Number of rows: 10
                         Limit Vectorization:
                             className: VectorLimitOperator
                             native: true
-                        Statistics: Num rows: 10 Data size: 4784 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 10 Data size: 4704 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
                           File Sink Vectorization:
                               className: VectorFileSinkOperator
                               native: false
-                          Statistics: Num rows: 10 Data size: 4784 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 10 Data size: 4704 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -151,8 +151,8 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypes_small
-                  filterExpr: (cboolean1 is not null and cint is not null and cdouble is not null and ctimestamp1 is not null) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 638316 Basic stats: COMPLETE Column stats: COMPLETE
+                  filterExpr: (cboolean1 is not null and cint is not null and ctimestamp1 is not null and cdouble is not null) (type: boolean)
+                  Statistics: Num rows: 12288 Data size: 513756 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -160,9 +160,9 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 10:boolean), SelectColumnIsNotNull(col 2:int), SelectColumnIsNotNull(col 5:double), SelectColumnIsNotNull(col 8:timestamp))
-                    predicate: (cboolean1 is not null and cint is not null and cdouble is not null and ctimestamp1 is not null) (type: boolean)
-                    Statistics: Num rows: 5112 Data size: 265564 Basic stats: COMPLETE Column stats: COMPLETE
+                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 10:boolean), SelectColumnIsNotNull(col 2:int), SelectColumnIsNotNull(col 8:timestamp), SelectColumnIsNotNull(col 5:double))
+                    predicate: (cboolean1 is not null and cint is not null and ctimestamp1 is not null and cdouble is not null) (type: boolean)
+                    Statistics: Num rows: 3816 Data size: 159600 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cdouble (type: double), cint (type: int), cboolean1 (type: boolean), ctimestamp1 (type: timestamp), CAST( cdouble AS decimal(20,10)) (type: decimal(20,10)), CAST( cint AS decimal(23,14)) (type: decimal(23,14)), CAST( cboolean1 AS decimal(5,2)) (type: decimal(5,2)), CAST( ctimestamp1 AS decimal(15,0)) (type: decimal(15,0))
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
@@ -171,19 +171,19 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [5, 2, 10, 8, 13, 14, 15, 16]
                           selectExpressions: CastDoubleToDecimal(col 5:double) -> 13:decimal(20,10), CastLongToDecimal(col 2:int) -> 14:decimal(23,14), CastLongToDecimal(col 10:boolean) -> 15:decimal(5,2), CastTimestampToDecimal(col 8:timestamp) -> 16:decimal(15,0)
-                      Statistics: Num rows: 5112 Data size: 2410700 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3816 Data size: 1760976 Basic stats: COMPLETE Column stats: COMPLETE
                       Limit
                         Number of rows: 10
                         Limit Vectorization:
                             className: VectorLimitOperator
                             native: true
-                        Statistics: Num rows: 10 Data size: 4784 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 10 Data size: 4704 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
                           File Sink Vectorization:
                               className: VectorFileSinkOperator
                               native: false
-                          Statistics: Num rows: 10 Data size: 4784 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 10 Data size: 4704 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_interval_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out
index 479d139..30c867e 100644
--- a/ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out
@@ -207,7 +207,7 @@ STAGE PLANS:
                 TableScan
                   alias: vectortab_a_1korc
                   filterExpr: (dt is not null and CAST( ts AS DATE) is not null and s is not null) (type: boolean)
-                  Statistics: Num rows: 1000 Data size: 187480 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1000 Data size: 185480 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -216,7 +216,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 12:date), SelectColumnIsNotNull(col 14:date)(children: CastTimestampToDate(col 10:timestamp) -> 14:date), SelectColumnIsNotNull(col 8:string))
                     predicate: (dt is not null and CAST( ts AS DATE) is not null and s is not null) (type: boolean)
-                    Statistics: Num rows: 954 Data size: 178852 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 905 Data size: 167854 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: s (type: string), (dt - CAST( ts AS DATE)) (type: interval_day_time)
                       outputColumnNames: _col0, _col1
@@ -225,7 +225,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [8, 16]
                           selectExpressions: DateColSubtractDateColumn(col 12:date, col 15:date)(children: CastTimestampToDate(col 10:timestamp) -> 15:date) -> 16:interval_day_time
-                      Statistics: Num rows: 954 Data size: 101124 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 905 Data size: 95930 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -240,7 +240,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 34600 Data size: 6920000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 32218 Data size: 6443600 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: string), _col2 (type: string), _col1 (type: interval_day_time)
                           outputColumnNames: _col0, _col1, _col2
@@ -248,13 +248,13 @@ STAGE PLANS:
                               className: VectorSelectOperator
                               native: true
                               projectedOutputColumnNums: [8, 8, 16]
-                          Statistics: Num rows: 34600 Data size: 6920000 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 32218 Data size: 6443600 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
                             File Sink Vectorization:
                                 className: VectorFileSinkOperator
                                 native: false
-                            Statistics: Num rows: 34600 Data size: 6920000 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 32218 Data size: 6443600 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -275,7 +275,7 @@ STAGE PLANS:
                 TableScan
                   alias: vectortab_b_1korc
                   filterExpr: (dt is not null and CAST( ts AS DATE) is not null and s is not null) (type: boolean)
-                  Statistics: Num rows: 1000 Data size: 186864 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1000 Data size: 184664 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -284,7 +284,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 12:date), SelectColumnIsNotNull(col 14:date)(children: CastTimestampToDate(col 10:timestamp) -> 14:date), SelectColumnIsNotNull(col 8:string))
                     predicate: (dt is not null and CAST( ts AS DATE) is not null and s is not null) (type: boolean)
-                    Statistics: Num rows: 943 Data size: 176202 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 890 Data size: 164340 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: s (type: string), (dt - CAST( ts AS DATE)) (type: interval_day_time)
                       outputColumnNames: _col0, _col1
@@ -293,7 +293,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [8, 16]
                           selectExpressions: DateColSubtractDateColumn(col 12:date, col 15:date)(children: CastTimestampToDate(col 10:timestamp) -> 15:date) -> 16:interval_day_time
-                      Statistics: Num rows: 943 Data size: 99958 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 890 Data size: 94340 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: interval_day_time)
                         sort order: ++
@@ -302,7 +302,7 @@ STAGE PLANS:
                             className: VectorReduceSinkMultiKeyOperator
                             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: 943 Data size: 99958 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 890 Data size: 94340 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vector_outer_join1.q.out b/ql/src/test/results/clientpositive/llap/vector_outer_join1.q.out
index 97814ad..a8e20bb 100644
--- a/ql/src/test/results/clientpositive/llap/vector_outer_join1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_outer_join1.q.out
@@ -252,7 +252,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: c
-                  Statistics: Num rows: 15 Data size: 3745 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 15 Data size: 3545 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -263,7 +263,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]
-                    Statistics: Num rows: 15 Data size: 3745 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 15 Data size: 3545 Basic stats: COMPLETE Column stats: COMPLETE
                     Map Join Operator
                       condition map:
                            Left Outer Join 0 to 1
@@ -284,13 +284,13 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
                       input vertices:
                         1 Map 2
-                      Statistics: Num rows: 33 Data size: 14459 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 33 Data size: 14099 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
                         File Sink Vectorization:
                             className: VectorFileSinkOperator
                             native: false
-                        Statistics: Num rows: 33 Data size: 14459 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 33 Data size: 14099 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -317,7 +317,7 @@ STAGE PLANS:
                 TableScan
                   alias: cd
                   filterExpr: cint is not null (type: boolean)
-                  Statistics: Num rows: 15 Data size: 3745 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 15 Data size: 3545 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -327,7 +327,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: SelectColumnIsNotNull(col 2:int)
                     predicate: cint is not null (type: boolean)
-                    Statistics: Num rows: 10 Data size: 2640 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 10 Data size: 2520 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
@@ -335,7 +335,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]
-                      Statistics: Num rows: 10 Data size: 2640 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 10 Data size: 2520 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col2 (type: int)
                         sort order: +
@@ -346,7 +346,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: 0:tinyint, 1:smallint, 3:bigint, 4:float, 5:double, 6:string, 7:string, 8:timestamp, 9:timestamp, 10:boolean, 11:boolean
-                        Statistics: Num rows: 10 Data size: 2640 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 10 Data size: 2520 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
diff --git a/ql/src/test/results/clientpositive/llap/vector_reuse_scratchcols.q.out b/ql/src/test/results/clientpositive/llap/vector_reuse_scratchcols.q.out
index fb3c012..a9cf533 100644
--- a/ql/src/test/results/clientpositive/llap/vector_reuse_scratchcols.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_reuse_scratchcols.q.out
@@ -97,7 +97,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((UDFToFloat(csmallint) < cfloat) and (UDFToDouble(ctimestamp2) > -5.0D) and (cdouble <> UDFToDouble(cint))) or ((CAST( cbigint AS decimal(22,3)) <= -1.389) and (cstring2 <> 'a') and (CAST( cint AS decimal(13,3)) <> 79.553) and (cboolean2 <> cboolean1)) or (cbigint = 762L) or (cstring1 = 'a')) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2477130 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -107,7 +107,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleColumn(col 13:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 1:smallint) -> 13:float), FilterDoubleColGreaterDoubleScalar(col 14:double, val -5.0)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), FilterDoubleColNotEqualDoubleColumn(col 5:double, col 15:double)(children: CastLongToDouble(col 2:int) -> 15:double)), FilterExprAndExpr(c [...]
                     predicate: (((UDFToFloat(csmallint) < cfloat) and (UDFToDouble(ctimestamp2) > -5.0D) and (cdouble <> UDFToDouble(cint))) or ((CAST( cbigint AS decimal(22,3)) <= -1.389) and (cstring2 <> 'a') and (CAST( cint AS decimal(13,3)) <> 79.553) and (cboolean2 <> cboolean1)) or (cbigint = 762L) or (cstring1 = 'a')) (type: boolean)
-                    Statistics: Num rows: 5465 Data size: 1157230 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 5465 Data size: 1101870 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int), cdouble (type: double), csmallint (type: smallint), cfloat (type: float), ctinyint (type: tinyint), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble(cint)) (type: double), UDFToDouble(csmallint) (type: double), (UDFToDouble(csmallint) * UDFToDouble(csmallint)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -116,7 +116,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [2, 5, 1, 4, 0, 18, 21, 22, 25]
                           selectExpressions: CastLongToDouble(col 2:int) -> 18:double, DoubleColMultiplyDoubleColumn(col 19:double, col 20:double)(children: CastLongToDouble(col 2:int) -> 19:double, CastLongToDouble(col 2:int) -> 20:double) -> 21:double, CastLongToDouble(col 1:smallint) -> 22:double, DoubleColMultiplyDoubleColumn(col 23:double, col 24:double)(children: CastLongToDouble(col 1:smallint) -> 23:double, CastLongToDouble(col 1:smallint) -> 24:double) -> 25:double
-                      Statistics: Num rows: 5465 Data size: 1157230 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 5465 Data size: 1101870 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0), count(_col0), sum(_col1), sum(_col6), sum(_col5), sum(_col8), sum(_col7), count(_col2), sum(_col3), count(_col3), min(_col4)
                         Group By Vectorization:
@@ -309,7 +309,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((UDFToFloat(csmallint) < cfloat) and (UDFToDouble(ctimestamp2) > -5.0D) and (cdouble <> UDFToDouble(cint))) or ((CAST( cbigint AS decimal(22,3)) <= -1.389) and (cstring2 <> 'a') and (CAST( cint AS decimal(13,3)) <> 79.553) and (cboolean2 <> cboolean1)) or (cbigint = 762L) or (cstring1 = 'a')) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2477130 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -319,7 +319,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleColumn(col 13:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 1:smallint) -> 13:float), FilterDoubleColGreaterDoubleScalar(col 14:double, val -5.0)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), FilterDoubleColNotEqualDoubleColumn(col 5:double, col 15:double)(children: CastLongToDouble(col 2:int) -> 15:double)), FilterExprAndExpr(c [...]
                     predicate: (((UDFToFloat(csmallint) < cfloat) and (UDFToDouble(ctimestamp2) > -5.0D) and (cdouble <> UDFToDouble(cint))) or ((CAST( cbigint AS decimal(22,3)) <= -1.389) and (cstring2 <> 'a') and (CAST( cint AS decimal(13,3)) <> 79.553) and (cboolean2 <> cboolean1)) or (cbigint = 762L) or (cstring1 = 'a')) (type: boolean)
-                    Statistics: Num rows: 5465 Data size: 1157230 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 5465 Data size: 1101870 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int), cdouble (type: double), csmallint (type: smallint), cfloat (type: float), ctinyint (type: tinyint), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble(cint)) (type: double), UDFToDouble(csmallint) (type: double), (UDFToDouble(csmallint) * UDFToDouble(csmallint)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -328,7 +328,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [2, 5, 1, 4, 0, 18, 21, 22, 25]
                           selectExpressions: CastLongToDouble(col 2:int) -> 18:double, DoubleColMultiplyDoubleColumn(col 19:double, col 20:double)(children: CastLongToDouble(col 2:int) -> 19:double, CastLongToDouble(col 2:int) -> 20:double) -> 21:double, CastLongToDouble(col 1:smallint) -> 22:double, DoubleColMultiplyDoubleColumn(col 23:double, col 24:double)(children: CastLongToDouble(col 1:smallint) -> 23:double, CastLongToDouble(col 1:smallint) -> 24:double) -> 25:double
-                      Statistics: Num rows: 5465 Data size: 1157230 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 5465 Data size: 1101870 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0), count(_col0), sum(_col1), sum(_col6), sum(_col5), sum(_col8), sum(_col7), count(_col2), sum(_col3), count(_col3), min(_col4)
                         Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_0.q.out b/ql/src/test/results/clientpositive/llap/vectorization_0.q.out
index 56faf2c..824d190 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_0.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_0.q.out
@@ -30102,7 +30102,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cint = 49) and (cfloat = 3.5)) or ((cint = 47) and (cfloat = 2.09)) or ((cint = 45) and (cfloat = 3.02))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
@@ -30227,7 +30227,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (struct(cint,cfloat)) IN (const struct(49,3.5), const struct(47,2.09), const struct(45,3.02)) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
@@ -30352,7 +30352,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cint = 49) or (cfloat = 3.5)) and ((cint = 47) or (cfloat = 2.09)) and ((cint = 45) or (cfloat = 3.02))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_10.q.out b/ql/src/test/results/clientpositive/llap/vectorization_10.q.out
index 6787949..761e1f5 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_10.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_10.q.out
@@ -68,7 +68,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cstring2 <= '10') or ((UDFToDouble(ctinyint) > cdouble) and (CAST( ctinyint AS decimal(6,2)) <= -5638.15)) or ((cdouble > 6981.0D) and ((CAST( csmallint AS decimal(11,4)) = 9763215.5639) or (cstring1 like '%a')))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -78,7 +78,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterStringGroupColLessEqualStringScalar(col 7:string, val 10), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleColumn(col 13:double, col 5:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterDecimalColLessEqualDecimalScalar(col 14:decimal(6,2), val -5638.15)(children: CastLongToDecimal(col 0:tinyint) -> 14:decimal(6,2))), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col  [...]
                     predicate: ((cstring2 <= '10') or ((UDFToDouble(ctinyint) > cdouble) and (CAST( ctinyint AS decimal(6,2)) <= -5638.15)) or ((cdouble > 6981.0D) and ((CAST( csmallint AS decimal(11,4)) = 9763215.5639) or (cstring1 like '%a')))) (type: boolean)
-                    Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cdouble (type: double), ctimestamp1 (type: timestamp), ctinyint (type: tinyint), cboolean1 (type: boolean), cstring1 (type: string), (- cdouble) (type: double), (cdouble + UDFToDouble(csmallint)) (type: double), ((cdouble + UDFToDouble(csmallint)) % 33.0D) (type: double), (- cdouble) (type: double), (UDFToDouble(ctinyint) % cdouble) (type: double), (UDFToShort(ctinyint) % csmallint) (type: smallint), (- cdouble) (type: double), (cbigint * UDFToLong((UDF [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -87,13 +87,13 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [5, 8, 0, 10, 6, 16, 18, 21, 22, 24, 25, 26, 28, 31, 33]
                           selectExpressions: DoubleColUnaryMinus(col 5:double) -> 16:double, DoubleColAddDoubleColumn(col 5:double, col 17:double)(children: CastLongToDouble(col 1:smallint) -> 17:double) -> 18:double, DoubleColModuloDoubleScalar(col 20:double, val 33.0)(children: DoubleColAddDoubleColumn(col 5:double, col 19:double)(children: CastLongToDouble(col 1:smallint) -> 19:double) -> 20:double) -> 21:double, DoubleColUnaryMinus(col 5:double) -> 22:double, DoubleColModuloDoubleCol [...]
-                      Statistics: Num rows: 12288 Data size: 2434654 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 12288 Data size: 2310094 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
                         File Sink Vectorization:
                             className: VectorFileSinkOperator
                             native: false
-                        Statistics: Num rows: 12288 Data size: 2434654 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 12288 Data size: 2310094 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/vectorization_11.q.out b/ql/src/test/results/clientpositive/llap/vectorization_11.q.out
index ff03d60..b50cab1 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_11.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_11.q.out
@@ -50,7 +50,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((ctimestamp1 is null and (cstring1 like '%a')) or (cstring2 = cstring1)) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2381474 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2256914 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -60,7 +60,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: SelectColumnIsNull(col 8:timestamp), FilterStringColLikeStringScalar(col 6:string, pattern %a)), FilterStringGroupColEqualStringGroupColumn(col 7:string, col 6:string))
                     predicate: ((ctimestamp1 is null and (cstring1 like '%a')) or (cstring2 = cstring1)) (type: boolean)
-                    Statistics: Num rows: 6144 Data size: 1190792 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7701 Data size: 1414500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cstring1 (type: string), cboolean1 (type: boolean), cdouble (type: double), ctimestamp1 (type: timestamp), (-3728 * UDFToInteger(csmallint)) (type: int), (cdouble - 9763215.5639D) (type: double), (- cdouble) (type: double), ((- cdouble) + 6981.0D) (type: double), (cdouble * -5638.15D) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -69,13 +69,13 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [6, 10, 5, 8, 13, 14, 15, 17, 18]
                           selectExpressions: LongScalarMultiplyLongColumn(val -3728, col 1:int)(children: col 1:smallint) -> 13:int, DoubleColSubtractDoubleScalar(col 5:double, val 9763215.5639) -> 14:double, DoubleColUnaryMinus(col 5:double) -> 15:double, DoubleColAddDoubleScalar(col 16:double, val 6981.0)(children: DoubleColUnaryMinus(col 5:double) -> 16:double) -> 17:double, DoubleColMultiplyDoubleScalar(col 5:double, val -5638.15) -> 18:double
-                      Statistics: Num rows: 6144 Data size: 953272 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7701 Data size: 1116736 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
                         File Sink Vectorization:
                             className: VectorFileSinkOperator
                             native: false
-                        Statistics: Num rows: 6144 Data size: 953272 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7701 Data size: 1116736 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/vectorization_12.q.out b/ql/src/test/results/clientpositive/llap/vectorization_12.q.out
index 620bc71..c7ba1d7 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_12.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_12.q.out
@@ -88,7 +88,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cstring1 like '%a') or ((cboolean2 <= 1) and (cbigint >= UDFToLong(csmallint)))) and ((cboolean1 >= cboolean2) or (UDFToShort(ctinyint) <> csmallint)) and ctimestamp1 is null) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 1647554 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1522994 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -98,7 +98,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterStringColLikeStringScalar(col 6:string, pattern %a), FilterExprAndExpr(children: FilterLongColLessEqualLongScalar(col 11:boolean, val 1), FilterLongColGreaterEqualLongColumn(col 3:bigint, col 1:bigint)(children: col 1:smallint))), FilterExprOrExpr(children: FilterLongColGreaterEqualLongColumn(col 10:boolean, col 11:boolean), FilterLongColNotEqualLongColumn(col 0:smallint, col 1:small [...]
                     predicate: (((cstring1 like '%a') or ((cboolean2 <= 1) and (cbigint >= UDFToLong(csmallint)))) and ((cboolean1 >= cboolean2) or (UDFToShort(ctinyint) <> csmallint)) and ctimestamp1 is null) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 166 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1903 Data size: 236052 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cbigint (type: bigint), cboolean1 (type: boolean), cstring1 (type: string), cdouble (type: double), UDFToDouble(cbigint) (type: double), (UDFToDouble(cbigint) * UDFToDouble(cbigint)) (type: double), (cdouble * cdouble) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
@@ -107,7 +107,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [3, 10, 6, 5, 13, 16, 17]
                           selectExpressions: CastLongToDouble(col 3:bigint) -> 13:double, DoubleColMultiplyDoubleColumn(col 14:double, col 15:double)(children: CastLongToDouble(col 3:bigint) -> 14:double, CastLongToDouble(col 3:bigint) -> 15:double) -> 16:double, DoubleColMultiplyDoubleColumn(col 5:double, col 5:double) -> 17:double
-                      Statistics: Num rows: 1 Data size: 166 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1903 Data size: 236052 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count(_col0), sum(_col5), sum(_col4), sum(_col3), count(_col3), sum(_col0), sum(_col6)
                         Group By Vectorization:
@@ -122,7 +122,7 @@ STAGE PLANS:
                         minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-                        Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1903 Data size: 268676 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: double), _col1 (type: bigint), _col2 (type: string), _col3 (type: boolean)
                           sort order: ++++
@@ -133,7 +133,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: 4:bigint, 5:double, 6:double, 7:double, 8:bigint, 9:bigint, 10:double
-                          Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1903 Data size: 268676 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col4 (type: bigint), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: bigint), _col9 (type: bigint), _col10 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -181,7 +181,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: double), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: boolean)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-                Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1903 Data size: 268676 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: bigint), _col3 (type: boolean), _col2 (type: string), _col0 (type: double), (-6432.0D * _col0) (type: double), (- _col1) (type: bigint), _col4 (type: bigint), (_col1 * _col4) (type: bigint), power(((_col5 - ((_col6 * _col6) / _col4)) / CASE WHEN ((_col4 = 1L)) THEN (null) ELSE ((_col4 - 1)) END), 0.5) (type: double), ((-6432.0D * _col0) / -6432.0D) (type: double), (- ((-6432.0D * _col0) / -6432.0D)) (type: double), (_col7 / _col8) (type: double [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col17, _col18, _col19
@@ -190,7 +190,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [1, 3, 2, 0, 11, 12, 4, 13, 21, 23, 26, 27, 29, 31, 9, 34, 38, 43, 48]
                       selectExpressions: DoubleScalarMultiplyDoubleColumn(val -6432.0, col 0:double) -> 11:double, LongColUnaryMinus(col 1:bigint) -> 12:bigint, LongColMultiplyLongColumn(col 1:bigint, col 4:bigint) -> 13:bigint, FuncPowerDoubleToDouble(col 20:double)(children: DoubleColDivideLongColumn(col 16:double, col 19:bigint)(children: DoubleColSubtractDoubleColumn(col 5:double, col 15:double)(children: DoubleColDivideLongColumn(col 14:double, col 4:bigint)(children: DoubleColMulti [...]
-                  Statistics: Num rows: 1 Data size: 346 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1903 Data size: 603604 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col3 (type: double), _col0 (type: bigint), _col2 (type: string)
                     sort order: +++
@@ -200,7 +200,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: 3:boolean, 11:double, 12:bigint, 4:bigint, 13:bigint, 21:double, 23:double, 26:double, 27:double, 29:double, 31:decimal(22,2), 9:bigint, 34:double, 38:double, 43:double, 48:double
-                    Statistics: Num rows: 1 Data size: 346 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1903 Data size: 603604 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: boolean), _col4 (type: double), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: double), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: double), _col13 (type: decimal(22,2)), _col14 (type: bigint), _col15 (type: double), _col17 (type: double), _col18 (type: double), _col19 (type: double)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -226,13 +226,13 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [1, 3, 2, 19, 0, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 11, 16, 17, 18]
                     selectExpressions: ConstantVectorExpression(val null) -> 19:timestamp
-                Statistics: Num rows: 1 Data size: 386 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1903 Data size: 603644 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 1 Data size: 386 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1903 Data size: 603644 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/vectorization_13.q.out b/ql/src/test/results/clientpositive/llap/vectorization_13.q.out
index e1d3032..6af2b75 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_13.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_13.q.out
@@ -90,7 +90,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28789.0D) and (UDFToDouble(ctimestamp2) <> -28788.0D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2028982 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1779902 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -100,7 +100,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val 3569.0), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 10.175), FilterLongColNotEqualLongScalar(col 10:boolean, val 1)), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28789.0)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val -2 [...]
                     predicate: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28789.0D) and (UDFToDouble(ctimestamp2) <> -28788.0D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-                    Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -109,7 +109,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [10, 0, 8, 4, 6, 4, 16, 17, 20]
                           selectExpressions: DoubleColMultiplyDoubleColumn(col 4:double, col 4:double)(children: col 4:float, col 4:float) -> 16:double, CastLongToDouble(col 0:tinyint) -> 17:double, DoubleColMultiplyDoubleColumn(col 18:double, col 19:double)(children: CastLongToDouble(col 0:tinyint) -> 18:double, CastLongToDouble(col 0:tinyint) -> 19:double) -> 20:double
-                      Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: max(_col1), sum(_col3), sum(_col6), sum(_col5), count(_col3), sum(_col8), sum(_col7), count(_col1), max(_col3), min(_col1)
                         Group By Vectorization:
@@ -121,10 +121,10 @@ STAGE PLANS:
                             vectorProcessingMode: HASH
                             projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9]
                         keys: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
-                        minReductionHashAggr: 0.99
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-                        Statistics: Num rows: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
                           sort order: +++++
@@ -135,7 +135,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: 5:tinyint, 6:double, 7:double, 8:double, 9:bigint, 10:double, 11:double, 12:bigint, 13:float, 14:tinyint
-                          Statistics: Num rows: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: double), _col8 (type: double), _col9 (type: bigint), _col10 (type: double), _col11 (type: double), _col12 (type: bigint), _col13 (type: float), _col14 (type: tinyint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -183,7 +183,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-                Statistics: Num rows: 346 Data size: 64822 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * _col3) (type: float), power(((_col7 - ((_col8 * _col8) / _col9)) / _col9), 0.5) (type: double), (- _col6) (type: double), power(((_col10 - ((_c [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
@@ -192,7 +192,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 1, 2, 3, 4, 15, 5, 17, 6, 21, 22, 23, 28, 29, 34, 38, 40, 43, 13, 49, 14]
                       selectExpressions: LongColUnaryMinus(col 1:tinyint) -> 15:tinyint, LongColAddLongColumn(col 16:tinyint, col 5:tinyint)(children: LongColUnaryMinus(col 1:tinyint) -> 16:tinyint) -> 17:tinyint, DoubleColMultiplyDoubleColumn(col 6:double, col 20:double)(children: CastLongToDouble(col 19:tinyint)(children: LongColAddLongColumn(col 18:tinyint, col 5:tinyint)(children: LongColUnaryMinus(col 1:tinyint) -> 18:tinyint) -> 19:tinyint) -> 20:double) -> 21:double, DoubleColUnar [...]
-                  Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), _col5 (type: tinyint), _col6 (type: tinyint), _col7 (type: tinyint), _col8 (type: double), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: double), _col13 (type: double), _col14 (type: double), _col15 (type: decimal(7,3)), _col16 (type: double), _col17 (type: double), _col18 (type: float), _col19 (type: dou [...]
                     sort order: +++++++++++++++++++++
@@ -201,7 +201,7 @@ STAGE PLANS:
                         keyColumns: 0:boolean, 1:tinyint, 2:timestamp, 3:float, 4:string, 15:tinyint, 5:tinyint, 17:tinyint, 6:double, 21:double, 22:double, 23:float, 28:double, 29:double, 34:double, 38:decimal(7,3), 40:double, 43:double, 13:float, 49:double, 14:tinyint
                         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: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.1
         Reducer 3 
             Execution mode: vectorized, llap
@@ -226,19 +226,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 10, 14, 15, 16, 17, 18, 19, 20]
-                Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 40
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 40 Data size: 12846 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 40 Data size: 12846 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -450,7 +450,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28801.388D) and (UDFToDouble(ctimestamp2) <> -28801.336D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2028982 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1779902 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -459,7 +459,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val 3569.0), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 10.175), FilterLongColNotEqualLongScalar(col 10:boolean, val 1)), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28801.388)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val  [...]
                     predicate: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28801.388D) and (UDFToDouble(ctimestamp2) <> -28801.336D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-                    Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -468,7 +468,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [10, 0, 8, 4, 6, 4, 16, 17, 20]
                           selectExpressions: DoubleColMultiplyDoubleColumn(col 4:double, col 4:double)(children: col 4:float, col 4:float) -> 16:double, CastLongToDouble(col 0:tinyint) -> 17:double, DoubleColMultiplyDoubleColumn(col 18:double, col 19:double)(children: CastLongToDouble(col 0:tinyint) -> 18:double, CastLongToDouble(col 0:tinyint) -> 19:double) -> 20:double
-                      Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: max(_col1), sum(_col3), sum(_col6), sum(_col5), count(_col3), sum(_col8), sum(_col7), count(_col1), max(_col3), min(_col1)
                         Group By Vectorization:
@@ -480,10 +480,10 @@ STAGE PLANS:
                             vectorProcessingMode: HASH
                             projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9]
                         keys: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
-                        minReductionHashAggr: 0.99
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-                        Statistics: Num rows: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
                           sort order: +++++
@@ -492,7 +492,7 @@ STAGE PLANS:
                               className: VectorReduceSinkMultiKeyOperator
                               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: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: double), _col8 (type: double), _col9 (type: bigint), _col10 (type: double), _col11 (type: double), _col12 (type: bigint), _col13 (type: float), _col14 (type: tinyint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -527,7 +527,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-                Statistics: Num rows: 346 Data size: 64822 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * _col3) (type: float), power(((_col7 - ((_col8 * _col8) / _col9)) / _col9), 0.5) (type: double), (- _col6) (type: double), power(((_col10 - ((_c [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
@@ -536,7 +536,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 1, 2, 3, 4, 15, 5, 17, 6, 21, 22, 23, 28, 29, 34, 38, 40, 43, 13, 49, 14]
                       selectExpressions: LongColUnaryMinus(col 1:tinyint) -> 15:tinyint, LongColAddLongColumn(col 16:tinyint, col 5:tinyint)(children: LongColUnaryMinus(col 1:tinyint) -> 16:tinyint) -> 17:tinyint, DoubleColMultiplyDoubleColumn(col 6:double, col 20:double)(children: CastLongToDouble(col 19:tinyint)(children: LongColAddLongColumn(col 18:tinyint, col 5:tinyint)(children: LongColUnaryMinus(col 1:tinyint) -> 18:tinyint) -> 19:tinyint) -> 20:double) -> 21:double, DoubleColUnar [...]
-                  Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), _col5 (type: tinyint), _col6 (type: tinyint), _col7 (type: tinyint), _col8 (type: double), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: double), _col13 (type: double), _col14 (type: double), _col15 (type: decimal(7,3)), _col16 (type: double), _col17 (type: double), _col18 (type: float), _col19 (type: dou [...]
                     sort order: +++++++++++++++++++++
@@ -544,7 +544,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: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.1
         Reducer 3 
             Execution mode: vectorized, llap
@@ -562,19 +562,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 10, 14, 15, 16, 17, 18, 19, 20]
-                Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 40
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 40 Data size: 12846 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 40 Data size: 12846 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/vectorization_14.q.out b/ql/src/test/results/clientpositive/llap/vectorization_14.q.out
index 1f7084a..ff5d42e 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_14.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_14.q.out
@@ -90,7 +90,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((UDFToLong(ctinyint) <= cbigint) and (cdouble < UDFToDouble(ctinyint)) and ((cbigint > -257L) or (cfloat < UDFToFloat(cint))) and ((UDFToDouble(cint) <= cdouble) or (ctimestamp2 < ctimestamp1))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2139070 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1889990 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -100,7 +100,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterLongColLessEqualLongColumn(col 0:bigint, col 3:bigint)(children: col 0:tinyint), FilterDoubleColLessDoubleColumn(col 5:double, col 13:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterExprOrExpr(children: FilterLongColGreaterLongScalar(col 3:bigint, val -257), FilterDoubleColLessDoubleColumn(col 4:float, col 14:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 14:float)), Filte [...]
                     predicate: ((UDFToLong(ctinyint) <= cbigint) and (cdouble < UDFToDouble(ctinyint)) and ((cbigint > -257L) or (cfloat < UDFToFloat(cint))) and ((UDFToDouble(cint) <= cdouble) or (ctimestamp2 < ctimestamp1))) (type: boolean)
-                    Statistics: Num rows: 758 Data size: 132082 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 758 Data size: 116802 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string), cboolean1 (type: boolean), cdouble (type: double), (- (-26.28D + cdouble)) (type: double), ((- (-26.28D + cdouble)) * (- (-26.28D + cdouble))) (type: double), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -109,7 +109,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [8, 4, 6, 10, 5, 17, 22, 4, 23]
                           selectExpressions: DoubleColUnaryMinus(col 16:double)(children: DoubleScalarAddDoubleColumn(val -26.28, col 5:double) -> 16:double) -> 17:double, DoubleColMultiplyDoubleColumn(col 19:double, col 21:double)(children: DoubleColUnaryMinus(col 18:double)(children: DoubleScalarAddDoubleColumn(val -26.28, col 5:double) -> 18:double) -> 19:double, DoubleColUnaryMinus(col 20:double)(children: DoubleScalarAddDoubleColumn(val -26.28, col 5:double) -> 20:double) -> 21:doub [...]
-                      Statistics: Num rows: 758 Data size: 132082 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 758 Data size: 116802 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col6), sum(_col5), count(_col5), max(_col1), sum(_col8), sum(_col7), count(_col1)
                         Group By Vectorization:
@@ -121,10 +121,10 @@ STAGE PLANS:
                             vectorProcessingMode: HASH
                             projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6]
                         keys: _col2 (type: string), _col1 (type: float), _col4 (type: double), _col0 (type: timestamp), _col3 (type: boolean)
-                        minReductionHashAggr: 0.99
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                        Statistics: Num rows: 379 Data size: 66108 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 379 Data size: 62308 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: float), _col2 (type: double), _col3 (type: timestamp), _col4 (type: boolean)
                           sort order: +++++
@@ -135,7 +135,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: 5:double, 6:double, 7:bigint, 8:float, 9:double, 10:double, 11:bigint
-                          Statistics: Num rows: 379 Data size: 66108 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 379 Data size: 62308 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col5 (type: double), _col6 (type: double), _col7 (type: bigint), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -183,7 +183,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: float), KEY._col2 (type: double), KEY._col3 (type: timestamp), KEY._col4 (type: boolean)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 189 Data size: 33008 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 379 Data size: 62308 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col3 (type: timestamp), _col1 (type: float), _col0 (type: string), _col4 (type: boolean), _col2 (type: double), (-26.28D + _col2) (type: double), (- (-26.28D + _col2)) (type: double), power(((_col5 - ((_col6 * _col6) / _col7)) / CASE WHEN ((_col7 = 1L)) THEN (null) ELSE ((_col7 - 1)) END), 0.5) (type: double), (_col1 * -26.28) (type: float), _col8 (type: float), (- _col1) (type: float), (- _col8) (type: float), ((- (-26.28D + _col2)) / 10.175D) (type: doub [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21
@@ -192,7 +192,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [3, 1, 0, 4, 2, 12, 14, 22, 23, 8, 24, 25, 28, 33, 11, 37, 46, 47, 51, 56, 63, 65]
                       selectExpressions: DoubleScalarAddDoubleColumn(val -26.28, col 2:double) -> 12:double, DoubleColUnaryMinus(col 13:double)(children: DoubleScalarAddDoubleColumn(val -26.28, col 2:double) -> 13:double) -> 14:double, FuncPowerDoubleToDouble(col 21:double)(children: DoubleColDivideLongColumn(col 17:double, col 20:bigint)(children: DoubleColSubtractDoubleColumn(col 5:double, col 16:double)(children: DoubleColDivideLongColumn(col 15:double, col 7:bigint)(children: DoubleC [...]
-                  Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 379 Data size: 88080 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: float), _col4 (type: double), _col0 (type: timestamp)
                     sort order: ++++
@@ -202,7 +202,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: 4:boolean, 12:double, 14:double, 22:double, 23:float, 8:float, 24:float, 25:float, 28:double, 33:double, 11:bigint, 37:double, 46:double, 47:double, 51:double, 56:double, 63:double, 65:double
-                    Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 379 Data size: 88080 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col3 (type: boolean), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: float), _col10 (type: float), _col11 (type: float), _col12 (type: double), _col13 (type: double), _col14 (type: bigint), _col15 (type: double), _col16 (type: double), _col17 (type: double), _col18 (type: double), _col19 (type: double), _col20 (type: double), _col21 (type: double)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -227,13 +227,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [3, 1, 0, 4, 2, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21]
-                Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 379 Data size: 88080 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 379 Data size: 88080 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/vectorization_15.q.out b/ql/src/test/results/clientpositive/llap/vectorization_15.q.out
index f7423ef..c176b0a 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_15.q.out
@@ -86,7 +86,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cstring1 like '10%') or (cstring2 like '%ss%') or ((cint >= -75) and (UDFToShort(ctinyint) = csmallint) and (cdouble >= -3728.0D))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -96,7 +96,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterStringColLikeStringScalar(col 6:string, pattern 10%), FilterStringColLikeStringScalar(col 7:string, pattern %ss%), FilterExprAndExpr(children: FilterLongColGreaterEqualLongScalar(col 2:int, val -75), FilterLongColEqualLongColumn(col 0:smallint, col 1:smallint)(children: col 0:tinyint), FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -3728.0)))
                     predicate: ((cstring1 like '10%') or (cstring2 like '%ss%') or ((cint >= -75) and (UDFToShort(ctinyint) = csmallint) and (cdouble >= -3728.0D))) (type: boolean)
-                    Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cfloat (type: float), cboolean1 (type: boolean), cdouble (type: double), cstring1 (type: string), ctinyint (type: tinyint), cint (type: int), ctimestamp1 (type: timestamp), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble(cint)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -105,7 +105,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [4, 10, 5, 6, 0, 2, 8, 4, 13, 14, 17, 18, 21]
                           selectExpressions: DoubleColMultiplyDoubleColumn(col 4:double, col 4:double)(children: col 4:float, col 4:float) -> 13:double, CastLongToDouble(col 0:tinyint) -> 14:double, DoubleColMultiplyDoubleColumn(col 15:double, col 16:double)(children: CastLongToDouble(col 0:tinyint) -> 15:double, CastLongToDouble(col 0:tinyint) -> 16:double) -> 17:double, CastLongToDouble(col 2:int) -> 18:double, DoubleColMultiplyDoubleColumn(col 19:double, col 20:double)(children: CastL [...]
-                      Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col8), sum(_col7), count(_col0), min(_col2), sum(_col10), sum(_col9), count(_col4), sum(_col12), sum(_col11), count(_col5)
                         Group By Vectorization:
@@ -117,10 +117,10 @@ STAGE PLANS:
                             vectorProcessingMode: HASH
                             projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9]
                         keys: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
-                        minReductionHashAggr: 0.99
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
-                        Statistics: Num rows: 6144 Data size: 1278652 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6144 Data size: 1216372 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
                           sort order: +++++++
@@ -131,7 +131,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: 7:double, 8:double, 9:bigint, 10:double, 11:double, 12:double, 13:bigint, 14:double, 15:double, 16:bigint
-                          Statistics: Num rows: 6144 Data size: 1278652 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 6144 Data size: 1216372 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col7 (type: double), _col8 (type: double), _col9 (type: bigint), _col10 (type: double), _col11 (type: double), _col12 (type: double), _col13 (type: bigint), _col14 (type: double), _col15 (type: double), _col16 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -162,15 +162,15 @@ STAGE PLANS:
                 keys: KEY._col0 (type: float), KEY._col1 (type: boolean), KEY._col2 (type: double), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int), KEY._col6 (type: timestamp)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
-                Statistics: Num rows: 3072 Data size: 639332 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 1216372 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp), power(((_col7 - ((_col8 * _col8) / _col9)) / CASE WHEN ((_col9 = 1L)) THEN (null) ELSE ((_col9 - 1)) END), 0.5) (type: double), (-26.28 - CAST( _col5 AS decimal(10,0))) (type: decimal(13,2)), _col10 (type: double), (_col2 * 79.553D) (type: double), (33.0 % _col0) (type: float), power(((_col11 - ((_col12 * [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-                  Statistics: Num rows: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6144 Data size: 2592628 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
                     sort order: +++++++
-                    Statistics: Num rows: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6144 Data size: 2592628 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col7 (type: double), _col8 (type: decimal(13,2)), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: double), _col13 (type: double), _col14 (type: double), _col15 (type: tinyint), _col16 (type: double), _col17 (type: float), _col18 (type: int), _col19 (type: decimal(13,2)), _col20 (type: double)
         Reducer 3 
             Execution mode: llap
@@ -182,10 +182,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: float), KEY.reducesinkkey1 (type: boolean), KEY.reducesinkkey2 (type: double), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: tinyint), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey6 (type: timestamp), VALUE._col0 (type: double), VALUE._col1 (type: decimal(13,2)), VALUE._col2 (type: double), VALUE._col3 (type: double), VALUE._col4 (type: float), VALUE._col5 (type: double), VALUE._col6 (type: double), VALUE._col7 (ty [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-                Statistics: Num rows: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 2592628 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6144 Data size: 2592628 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/vectorization_16.q.out b/ql/src/test/results/clientpositive/llap/vectorization_16.q.out
index e9e8d1f..01b98c3 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_16.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_16.q.out
@@ -63,7 +63,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2308074 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2183514 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -73,7 +73,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterStringColLikeStringScalar(col 7:string, pattern %b%), FilterExprOrExpr(children: FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -1.389), FilterStringGroupColLessStringScalar(col 6:string, val a)))
                     predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-                    Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp), (cdouble * cdouble) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3
@@ -82,7 +82,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [6, 5, 8, 13]
                           selectExpressions: DoubleColMultiplyDoubleColumn(col 5:double, col 5:double) -> 13:double
-                      Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count(_col1), sum(_col3), sum(_col1), min(_col1)
                         Group By Vectorization:
@@ -94,10 +94,10 @@ STAGE PLANS:
                             vectorProcessingMode: HASH
                             projectedOutputColumnNums: [0, 1, 2, 3]
                         keys: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
-                        minReductionHashAggr: 0.99
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                        Statistics: Num rows: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
                           sort order: +++
@@ -108,7 +108,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: 3:bigint, 4:double, 5:double, 6:double
-                          Statistics: Num rows: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col3 (type: bigint), _col4 (type: double), _col5 (type: double), _col6 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -156,7 +156,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                Statistics: Num rows: 1536 Data size: 227586 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639D) (type: double), (- (_col1 - 9763215.5639D)) (type: double), _col3 (type: bigint), power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5) (type: double), (- power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5)) (type: double), (power(((_col4 - ((_col5 * [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -165,13 +165,13 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 1, 2, 7, 9, 3, 17, 26, 36, 6, 37, 39, 47]
                       selectExpressions: DoubleColSubtractDoubleScalar(col 1:double, val 9763215.5639) -> 7:double, DoubleColUnaryMinus(col 8:double)(children: DoubleColSubtractDoubleScalar(col 1:double, val 9763215.5639) -> 8:double) -> 9:double, FuncPowerDoubleToDouble(col 16:double)(children: DoubleColDivideLongColumn(col 12:double, col 15:bigint)(children: DoubleColSubtractDoubleColumn(col 4:double, col 11:double)(children: DoubleColDivideLongColumn(col 10:double, col 3:bigint)(child [...]
-                  Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3072 Data size: 890996 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3072 Data size: 890996 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/vectorization_17.q.out b/ql/src/test/results/clientpositive/llap/vectorization_17.q.out
index 1b7dc7e..a4ab091 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_17.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_17.q.out
@@ -71,7 +71,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cbigint > -23L) and ((ctinyint >= 33Y) or (UDFToLong(csmallint) >= cbigint) or (UDFToDouble(cfloat) = cdouble)) and ((cdouble <> 988888.0D) or (CAST( cint AS decimal(13,3)) > -863.257))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 1647550 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1522990 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -81,7 +81,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterLongColGreaterLongScalar(col 3:bigint, val -23), FilterExprOrExpr(children: FilterLongColGreaterEqualLongScalar(col 0:tinyint, val 33), FilterLongColGreaterEqualLongColumn(col 1:bigint, col 3:bigint)(children: col 1:smallint), FilterDoubleColEqualDoubleColumn(col 4:double, col 5:double)(children: col 4:float)), FilterExprOrExpr(children: FilterDoubleColNotEqualDoubleScalar(col 5:double, val 988888.0), FilterDe [...]
                     predicate: ((cbigint > -23L) and ((ctinyint >= 33Y) or (UDFToLong(csmallint) >= cbigint) or (UDFToDouble(cfloat) = cdouble)) and ((cdouble <> 988888.0D) or (CAST( cint AS decimal(13,3)) > -863.257))) (type: boolean)
-                    Statistics: Num rows: 6141 Data size: 823456 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6141 Data size: 761216 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cfloat (type: float), cstring1 (type: string), cint (type: int), ctimestamp1 (type: timestamp), cdouble (type: double), cbigint (type: bigint), (UDFToDouble(cfloat) / UDFToDouble(ctinyint)) (type: double), (UDFToLong(cint) % cbigint) (type: bigint), (- cdouble) (type: double), (cdouble + (UDFToDouble(cfloat) / UDFToDouble(ctinyint))) (type: double), (cdouble / UDFToDouble(cint)) (type: double), (- (- cdouble)) (type: double), (9763215.5639 % CAST( cbigi [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -90,7 +90,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [4, 6, 2, 8, 5, 3, 15, 16, 17, 20, 22, 24, 26, 29]
                           selectExpressions: DoubleColDivideDoubleColumn(col 4:double, col 14:double)(children: col 4:float, CastLongToDouble(col 0:tinyint) -> 14:double) -> 15:double, LongColModuloLongColumn(col 2:bigint, col 3:bigint)(children: col 2:int) -> 16:bigint, DoubleColUnaryMinus(col 5:double) -> 17:double, DoubleColAddDoubleColumn(col 5:double, col 19:double)(children: DoubleColDivideDoubleColumn(col 4:double, col 18:double)(children: col 4:float, CastLongToDouble(col 0:tinyi [...]
-                      Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6141 Data size: 1756220 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col5 (type: bigint), _col0 (type: float)
                         sort order: ++
@@ -100,7 +100,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: 6:string, 2:int, 8:timestamp, 5:double, 15:double, 16:bigint, 17:double, 20:double, 22:double, 24:double, 26:decimal(11,4), 29:double
-                        Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6141 Data size: 1756220 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: timestamp), _col4 (type: double), _col6 (type: double), _col7 (type: bigint), _col8 (type: double), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: decimal(11,4)), _col13 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -142,13 +142,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [1, 2, 3, 4, 5, 0, 6, 7, 8, 9, 10, 11, 12, 13]
-                Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6141 Data size: 1756220 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6141 Data size: 1756220 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/vectorization_2.q.out b/ql/src/test/results/clientpositive/llap/vectorization_2.q.out
index 33c34a1..02d4fa5 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_2.q.out
@@ -69,7 +69,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cdouble < UDFToDouble(ctinyint)) and ((UDFToDouble(ctimestamp2) <> -10669.0D) or (cint < 359))) or ((ctimestamp1 < ctimestamp2) and (cstring2 like 'b%') and (cfloat <= -5638.15))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2157324 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1908244 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -79,7 +79,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleColumn(col 5:double, col 13:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterExprOrExpr(children: FilterDoubleColNotEqualDoubleScalar(col 14:double, val -10669.0)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), FilterLongColLessLongScalar(col 2:int, val 359))), FilterExprAndExpr(children: FilterTimestampColLessTimestampColumn(c [...]
                     predicate: (((cdouble < UDFToDouble(ctinyint)) and ((UDFToDouble(ctimestamp2) <> -10669.0D) or (cint < 359))) or ((ctimestamp1 < ctimestamp2) and (cstring2 like 'b%') and (cfloat <= -5638.15))) (type: boolean)
-                    Statistics: Num rows: 4096 Data size: 719232 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4096 Data size: 636272 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: csmallint (type: smallint), cfloat (type: float), cbigint (type: bigint), ctinyint (type: tinyint), cdouble (type: double), UDFToDouble(cbigint) (type: double), (UDFToDouble(cbigint) * UDFToDouble(cbigint)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
@@ -88,7 +88,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [1, 4, 3, 0, 5, 15, 18]
                           selectExpressions: CastLongToDouble(col 3:bigint) -> 15:double, DoubleColMultiplyDoubleColumn(col 16:double, col 17:double)(children: CastLongToDouble(col 3:bigint) -> 16:double, CastLongToDouble(col 3:bigint) -> 17:double) -> 18:double
-                      Statistics: Num rows: 4096 Data size: 719232 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4096 Data size: 636272 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0), count(_col0), sum(_col1), sum(_col6), sum(_col5), count(_col2), count(), min(_col3), sum(_col4), count(_col4)
                         Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_3.q.out b/ql/src/test/results/clientpositive/llap/vectorization_3.q.out
index 7d1cadc..10977ca 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_3.q.out
@@ -74,7 +74,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((UDFToFloat(cint) <= cfloat) and (CAST( cbigint AS decimal(22,3)) <> 79.553) and (UDFToDouble(ctimestamp2) = -29071.0D)) or ((UDFToDouble(cbigint) > cdouble) and (CAST( csmallint AS decimal(8,3)) >= 79.553) and (ctimestamp1 > ctimestamp2))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 1276620 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1027540 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -84,7 +84,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleColumn(col 13:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 13:float), FilterDecimalColNotEqualDecimalScalar(col 14:decimal(22,3), val 79.553)(children: CastLongToDecimal(col 3:bigint) -> 14:decimal(22,3)), FilterDoubleColEqualDoubleScalar(col 15:double, val -29071.0)(children: CastTimestampToDouble(col 9:timestamp) -> 15:double)), [...]
                     predicate: (((UDFToFloat(cint) <= cfloat) and (CAST( cbigint AS decimal(22,3)) <> 79.553) and (UDFToDouble(ctimestamp2) = -29071.0D)) or ((UDFToDouble(cbigint) > cdouble) and (CAST( csmallint AS decimal(8,3)) >= 79.553) and (ctimestamp1 > ctimestamp2))) (type: boolean)
-                    Statistics: Num rows: 2503 Data size: 260060 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2503 Data size: 209380 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: csmallint (type: smallint), ctinyint (type: tinyint), cfloat (type: float), cint (type: int), UDFToDouble(csmallint) (type: double), (UDFToDouble(csmallint) * UDFToDouble(csmallint)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFTo [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
@@ -93,7 +93,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [1, 0, 4, 2, 18, 21, 22, 25, 4, 26, 27, 30]
                           selectExpressions: CastLongToDouble(col 1:smallint) -> 18:double, DoubleColMultiplyDoubleColumn(col 19:double, col 20:double)(children: CastLongToDouble(col 1:smallint) -> 19:double, CastLongToDouble(col 1:smallint) -> 20:double) -> 21:double, CastLongToDouble(col 0:tinyint) -> 22:double, DoubleColMultiplyDoubleColumn(col 23:double, col 24:double)(children: CastLongToDouble(col 0:tinyint) -> 23:double, CastLongToDouble(col 0:tinyint) -> 24:double) -> 25:double,  [...]
-                      Statistics: Num rows: 2503 Data size: 260060 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2503 Data size: 209380 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col5), sum(_col4), count(_col0), sum(_col7), sum(_col6), count(_col1), sum(_col9), sum(_col8), count(_col2), sum(_col2), sum(_col3), count(_col3), sum(_col11), sum(_col10)
                         Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_5.q.out b/ql/src/test/results/clientpositive/llap/vectorization_5.q.out
index 836e8bb..db823ff 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_5.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_5.q.out
@@ -63,7 +63,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cstring1 like '%b%') and cboolean2 is not null) or ((UDFToDouble(ctinyint) = cdouble) and (cstring2 like 'a') and ctimestamp2 is not null)) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2454862 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2330342 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -73,7 +73,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterStringColLikeStringScalar(col 6:string, pattern %b%), SelectColumnIsNotNull(col 11:boolean)), FilterExprAndExpr(children: FilterDoubleColEqualDoubleColumn(col 13:double, col 5:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterStringColLikeStringScalar(col 7:string, pattern a), SelectColumnIsNotNull(col 9:timestamp)))
                     predicate: (((cstring1 like '%b%') and cboolean2 is not null) or ((UDFToDouble(ctinyint) = cdouble) and (cstring2 like 'a') and ctimestamp2 is not null)) (type: boolean)
-                    Statistics: Num rows: 7658 Data size: 1529972 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6879 Data size: 1304690 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int)
                       outputColumnNames: ctinyint, csmallint, cint
@@ -81,7 +81,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1, 2]
-                      Statistics: Num rows: 7658 Data size: 1529972 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6879 Data size: 1304690 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: max(csmallint), count(), min(csmallint), sum(cint), max(ctinyint)
                         Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_7.q.out b/ql/src/test/results/clientpositive/llap/vectorization_7.q.out
index 51ae4a0..8261276 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_7.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_7.q.out
@@ -77,7 +77,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28815.0D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3019778 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2770698 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -87,7 +87,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterDoubleColGreaterDoubleScalar(col 5:double, val 988888.0), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28815.0)(children: CastTimestampToDouble(col 9:timestamp) -> 13:double), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 3569.0))), FilterExprOrExpr(children: FilterDoubleColLessEqualDoubleScalar(col 14:double, val -28800.0)(children: Cas [...]
                     predicate: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28815.0D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-                    Statistics: Num rows: 11033 Data size: 2711364 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 11033 Data size: 2487724 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cboolean1 (type: boolean), cbigint (type: bigint), csmallint (type: smallint), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cstring1 (type: string), (cbigint + cbigint) (type: bigint), (UDFToInteger(csmallint) % -257) (type: int), (- csmallint) (type: smallint), (- ctinyint) (type: tinyint), (UDFToInteger((- ctinyint)) + 17) (type: int), (cbigint * UDFToLong((- csmallint))) (type: bigint), (cint % UDFToInteger(csmallint)) (type: int), (- cti [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -96,7 +96,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [10, 3, 1, 0, 8, 6, 15, 16, 17, 18, 20, 22, 23, 24, 26]
                           selectExpressions: LongColAddLongColumn(col 3:bigint, col 3:bigint) -> 15:bigint, LongColModuloLongScalar(col 1:int, val -257)(children: col 1:smallint) -> 16:int, LongColUnaryMinus(col 1:smallint) -> 17:smallint, LongColUnaryMinus(col 0:tinyint) -> 18:tinyint, LongColAddLongScalar(col 19:int, val 17)(children: LongColUnaryMinus(col 0:tinyint) -> 19:tinyint) -> 20:int, LongColMultiplyLongColumn(col 3:bigint, col 21:bigint)(children: LongColUnaryMinus(col 1:small [...]
-                      Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: boolean), _col1 (type: bigint), _col2 (type: smallint), _col3 (type: tinyint), _col4 (type: timestamp), _col5 (type: string), _col6 (type: bigint), _col7 (type: int), _col8 (type: smallint), _col9 (type: tinyint), _col10 (type: int), _col11 (type: bigint), _col12 (type: int), _col13 (type: tinyint), _col14 (type: tinyint)
                         sort order: +++++++++++++++
@@ -105,7 +105,7 @@ STAGE PLANS:
                             keyColumns: 10:boolean, 3:bigint, 1:smallint, 0:tinyint, 8:timestamp, 6:string, 15:bigint, 16:int, 17:smallint, 18:tinyint, 20:int, 22:bigint, 23:int, 24:tinyint, 26:tinyint
                             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: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -147,19 +147,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 9, 14]
-                Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 25
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 25 Data size: 4180 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 25 Data size: 4180 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -331,7 +331,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28792.315D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3019778 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2770698 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -340,7 +340,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterDoubleColGreaterDoubleScalar(col 5:double, val 988888.0), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28792.315)(children: CastTimestampToDouble(col 9:timestamp) -> 13:double), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 3569.0))), FilterExprOrExpr(children: FilterDoubleColLessEqualDoubleScalar(col 14:double, val -28800.0)(children: C [...]
                     predicate: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28792.315D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-                    Statistics: Num rows: 11033 Data size: 2711364 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 11033 Data size: 2487724 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cboolean1 (type: boolean), cbigint (type: bigint), csmallint (type: smallint), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cstring1 (type: string), (cbigint + cbigint) (type: bigint), (UDFToInteger(csmallint) % -257) (type: int), (- csmallint) (type: smallint), (- ctinyint) (type: tinyint), (UDFToInteger((- ctinyint)) + 17) (type: int), (cbigint * UDFToLong((- csmallint))) (type: bigint), (cint % UDFToInteger(csmallint)) (type: int), (- cti [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -349,7 +349,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [10, 3, 1, 0, 8, 6, 15, 16, 17, 18, 20, 22, 23, 24, 26]
                           selectExpressions: LongColAddLongColumn(col 3:bigint, col 3:bigint) -> 15:bigint, LongColModuloLongScalar(col 1:int, val -257)(children: col 1:smallint) -> 16:int, LongColUnaryMinus(col 1:smallint) -> 17:smallint, LongColUnaryMinus(col 0:tinyint) -> 18:tinyint, LongColAddLongScalar(col 19:int, val 17)(children: LongColUnaryMinus(col 0:tinyint) -> 19:tinyint) -> 20:int, LongColMultiplyLongColumn(col 3:bigint, col 21:bigint)(children: LongColUnaryMinus(col 1:small [...]
-                      Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: boolean), _col1 (type: bigint), _col2 (type: smallint), _col3 (type: tinyint), _col4 (type: timestamp), _col5 (type: string), _col6 (type: bigint), _col7 (type: int), _col8 (type: smallint), _col9 (type: tinyint), _col10 (type: int), _col11 (type: bigint), _col12 (type: int), _col13 (type: tinyint), _col14 (type: tinyint)
                         sort order: +++++++++++++++
@@ -357,7 +357,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: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -386,19 +386,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 9, 14]
-                Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 25
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 25 Data size: 4180 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 25 Data size: 4180 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/vectorization_8.q.out b/ql/src/test/results/clientpositive/llap/vectorization_8.q.out
index eaa1f4d..3c42e1d 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_8.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_8.q.out
@@ -73,7 +73,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 10.0D) and (UDFToDouble(ctimestamp2) <> 16.0D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2983078 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2733998 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -83,7 +83,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val -6432.0), FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleScalar(col 13:double, val 10.0)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val 16.0)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), SelectColumnIsNotNull(col 7:string)), FilterExprAndExpr(children: FilterDoubleColE [...]
                     predicate: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 10.0D) and (UDFToDouble(ctimestamp2) <> 16.0D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-                    Statistics: Num rows: 3059 Data size: 742850 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3059 Data size: 680930 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctimestamp1 (type: timestamp), cdouble (type: double), cboolean1 (type: boolean), cstring1 (type: string), cfloat (type: float), (- cdouble) (type: double), (-5638.15D - cdouble) (type: double), (cdouble * -257.0D) (type: double), (UDFToFloat(cint) + cfloat) (type: float), ((- cdouble) + UDFToDouble(cbigint)) (type: double), (- cdouble) (type: double), (-1.389 - cfloat) (type: float), (- cfloat) (type: float), ((-5638.15D - cdouble) + UDFToDouble((UDFTo [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -92,7 +92,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [8, 5, 10, 6, 4, 15, 16, 17, 19, 22, 23, 24, 25, 29]
                           selectExpressions: DoubleColUnaryMinus(col 5:double) -> 15:double, DoubleScalarSubtractDoubleColumn(val -5638.15, col 5:double) -> 16:double, DoubleColMultiplyDoubleScalar(col 5:double, val -257.0) -> 17:double, DoubleColAddDoubleColumn(col 18:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 18:float) -> 19:float, DoubleColAddDoubleColumn(col 20:double, col 21:double)(children: DoubleColUnaryMinus(col 5:double) -> 20:double, CastLongToD [...]
-                      Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: timestamp), _col1 (type: double), _col2 (type: boolean), _col3 (type: string), _col4 (type: float), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: float), _col13 (type: double)
                         sort order: ++++++++++++++
@@ -101,7 +101,7 @@ STAGE PLANS:
                             keyColumns: 8:timestamp, 5:double, 10:boolean, 6:string, 4:float, 15:double, 16:double, 17:double, 19:float, 22:double, 23:double, 24:float, 25:float, 29:double
                             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: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -143,19 +143,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 5, 11, 12, 13]
-                Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 20 Data size: 3600 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 20 Data size: 3600 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -314,7 +314,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 12.503D) and (UDFToDouble(ctimestamp2) <> 11.998D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2983078 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2733998 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -323,7 +323,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val -6432.0), FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleScalar(col 13:double, val 12.503)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val 11.998)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), SelectColumnIsNotNull(col 7:string)), FilterExprAndExpr(children: FilterDouble [...]
                     predicate: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 12.503D) and (UDFToDouble(ctimestamp2) <> 11.998D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-                    Statistics: Num rows: 3059 Data size: 742850 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3059 Data size: 680930 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctimestamp1 (type: timestamp), cdouble (type: double), cboolean1 (type: boolean), cstring1 (type: string), cfloat (type: float), (- cdouble) (type: double), (-5638.15D - cdouble) (type: double), (cdouble * -257.0D) (type: double), (UDFToFloat(cint) + cfloat) (type: float), ((- cdouble) + UDFToDouble(cbigint)) (type: double), (- cdouble) (type: double), (-1.389 - cfloat) (type: float), (- cfloat) (type: float), ((-5638.15D - cdouble) + UDFToDouble((UDFTo [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -332,7 +332,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [8, 5, 10, 6, 4, 15, 16, 17, 19, 22, 23, 24, 25, 29]
                           selectExpressions: DoubleColUnaryMinus(col 5:double) -> 15:double, DoubleScalarSubtractDoubleColumn(val -5638.15, col 5:double) -> 16:double, DoubleColMultiplyDoubleScalar(col 5:double, val -257.0) -> 17:double, DoubleColAddDoubleColumn(col 18:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 18:float) -> 19:float, DoubleColAddDoubleColumn(col 20:double, col 21:double)(children: DoubleColUnaryMinus(col 5:double) -> 20:double, CastLongToD [...]
-                      Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: timestamp), _col1 (type: double), _col2 (type: boolean), _col3 (type: string), _col4 (type: float), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: float), _col13 (type: double)
                         sort order: ++++++++++++++
@@ -340,7 +340,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: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -369,19 +369,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 5, 11, 12, 13]
-                Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 20 Data size: 3600 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 20 Data size: 3600 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/vectorization_9.q.out b/ql/src/test/results/clientpositive/llap/vectorization_9.q.out
index e9e8d1f..01b98c3 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_9.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_9.q.out
@@ -63,7 +63,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2308074 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2183514 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -73,7 +73,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterStringColLikeStringScalar(col 7:string, pattern %b%), FilterExprOrExpr(children: FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -1.389), FilterStringGroupColLessStringScalar(col 6:string, val a)))
                     predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-                    Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp), (cdouble * cdouble) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3
@@ -82,7 +82,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [6, 5, 8, 13]
                           selectExpressions: DoubleColMultiplyDoubleColumn(col 5:double, col 5:double) -> 13:double
-                      Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count(_col1), sum(_col3), sum(_col1), min(_col1)
                         Group By Vectorization:
@@ -94,10 +94,10 @@ STAGE PLANS:
                             vectorProcessingMode: HASH
                             projectedOutputColumnNums: [0, 1, 2, 3]
                         keys: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
-                        minReductionHashAggr: 0.99
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                        Statistics: Num rows: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
                           sort order: +++
@@ -108,7 +108,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: 3:bigint, 4:double, 5:double, 6:double
-                          Statistics: Num rows: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col3 (type: bigint), _col4 (type: double), _col5 (type: double), _col6 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -156,7 +156,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                Statistics: Num rows: 1536 Data size: 227586 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639D) (type: double), (- (_col1 - 9763215.5639D)) (type: double), _col3 (type: bigint), power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5) (type: double), (- power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5)) (type: double), (power(((_col4 - ((_col5 * [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -165,13 +165,13 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 1, 2, 7, 9, 3, 17, 26, 36, 6, 37, 39, 47]
                       selectExpressions: DoubleColSubtractDoubleScalar(col 1:double, val 9763215.5639) -> 7:double, DoubleColUnaryMinus(col 8:double)(children: DoubleColSubtractDoubleScalar(col 1:double, val 9763215.5639) -> 8:double) -> 9:double, FuncPowerDoubleToDouble(col 16:double)(children: DoubleColDivideLongColumn(col 12:double, col 15:bigint)(children: DoubleColSubtractDoubleColumn(col 4:double, col 11:double)(children: DoubleColDivideLongColumn(col 10:double, col 3:bigint)(child [...]
-                  Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3072 Data size: 890996 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3072 Data size: 890996 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/vectorization_short_regress.q.out b/ql/src/test/results/clientpositive/llap/vectorization_short_regress.q.out
index 8eb523d..39aaa56 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_short_regress.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_short_regress.q.out
@@ -97,7 +97,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((UDFToFloat(csmallint) < cfloat) and (UDFToDouble(ctimestamp2) > -5.0D) and (cdouble <> UDFToDouble(cint))) or ((CAST( cbigint AS decimal(22,3)) <= -1.389) and (cstring2 <> 'a') and (CAST( cint AS decimal(13,3)) <> 79.553) and (cboolean2 <> cboolean1)) or (cbigint = 762L) or (cstring1 = 'a')) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2477130 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -106,7 +106,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleColumn(col 13:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 1:smallint) -> 13:float), FilterDoubleColGreaterDoubleScalar(col 14:double, val -5.0)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), FilterDoubleColNotEqualDoubleColumn(col 5:double, col 15:double)(children: CastLongToDouble(col 2:int) -> 15:double)), FilterExprAndExpr(c [...]
                     predicate: (((UDFToFloat(csmallint) < cfloat) and (UDFToDouble(ctimestamp2) > -5.0D) and (cdouble <> UDFToDouble(cint))) or ((CAST( cbigint AS decimal(22,3)) <= -1.389) and (cstring2 <> 'a') and (CAST( cint AS decimal(13,3)) <> 79.553) and (cboolean2 <> cboolean1)) or (cbigint = 762L) or (cstring1 = 'a')) (type: boolean)
-                    Statistics: Num rows: 5465 Data size: 1157230 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 5465 Data size: 1101870 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int), cdouble (type: double), csmallint (type: smallint), cfloat (type: float), ctinyint (type: tinyint), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble(cint)) (type: double), UDFToDouble(csmallint) (type: double), (UDFToDouble(csmallint) * UDFToDouble(csmallint)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -115,7 +115,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [2, 5, 1, 4, 0, 18, 21, 22, 25]
                           selectExpressions: CastLongToDouble(col 2:int) -> 18:double, DoubleColMultiplyDoubleColumn(col 19:double, col 20:double)(children: CastLongToDouble(col 2:int) -> 19:double, CastLongToDouble(col 2:int) -> 20:double) -> 21:double, CastLongToDouble(col 1:smallint) -> 22:double, DoubleColMultiplyDoubleColumn(col 23:double, col 24:double)(children: CastLongToDouble(col 1:smallint) -> 23:double, CastLongToDouble(col 1:smallint) -> 24:double) -> 25:double
-                      Statistics: Num rows: 5465 Data size: 1157230 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 5465 Data size: 1101870 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0), count(_col0), sum(_col1), sum(_col6), sum(_col5), sum(_col8), sum(_col7), count(_col2), sum(_col3), count(_col3), min(_col4)
                         Group By Vectorization:
@@ -625,7 +625,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((ctimestamp1 = ctimestamp2) or ((UDFToLong(csmallint) <= cbigint) and (cboolean2 = 1)) or ((cstring2 > 'a') and cboolean1 is not null and ctimestamp2 is not null) or (cfloat = 762.0) or (cstring1 = 'ss')) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -634,7 +634,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterTimestampColEqualTimestampColumn(col 8:timestamp, col 9:timestamp), FilterExprAndExpr(children: FilterLongColLessEqualLongColumn(col 1:bigint, col 3:bigint)(children: col 1:smallint), FilterLongColEqualLongScalar(col 11:boolean, val 1)), FilterExprAndExpr(children: FilterStringGroupColGreaterStringScalar(col 7:string, val a), SelectColumnIsNotNull(col 10:boolean), SelectColumnIsNotNull(col 9:timestamp)), Filter [...]
                     predicate: ((ctimestamp1 = ctimestamp2) or ((UDFToLong(csmallint) <= cbigint) and (cboolean2 = 1)) or ((cstring2 > 'a') and cboolean1 is not null and ctimestamp2 is not null) or (cfloat = 762.0) or (cstring1 = 'ss')) (type: boolean)
-                    Statistics: Num rows: 11346 Data size: 2856120 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 10571 Data size: 2446670 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cbigint (type: bigint), ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cdouble (type: double), UDFToDouble(cbigint) (type: double), (UDFToDouble(cbigint) * UDFToDouble(cbigint)) (type: double), UDFToDouble(csmallint) (type: double), (UDFToDouble(csmallint) * UDFToDouble(csmallint)) (type: double), (cdouble * cdouble) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -643,7 +643,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [3, 0, 1, 2, 5, 13, 16, 17, 20, 21]
                           selectExpressions: CastLongToDouble(col 3:bigint) -> 13:double, DoubleColMultiplyDoubleColumn(col 14:double, col 15:double)(children: CastLongToDouble(col 3:bigint) -> 14:double, CastLongToDouble(col 3:bigint) -> 15:double) -> 16:double, CastLongToDouble(col 1:smallint) -> 17:double, DoubleColMultiplyDoubleColumn(col 18:double, col 19:double)(children: CastLongToDouble(col 1:smallint) -> 18:double, CastLongToDouble(col 1:smallint) -> 19:double) -> 20:double, Dou [...]
-                      Statistics: Num rows: 11346 Data size: 2856120 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 10571 Data size: 2446670 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col6), sum(_col5), count(_col0), count(), max(_col1), sum(_col8), sum(_col7), count(_col2), max(_col3), sum(_col9), sum(_col4), count(_col4), count(_col1), sum(_col1)
                         Group By Vectorization:
@@ -864,7 +864,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((csmallint < UDFToShort(ctinyint)) and (UDFToDouble(ctimestamp1) >= 0.0D)) or (cfloat = 17.0) or ((ctimestamp2 <= ctimestamp1) and (UDFToDouble(cbigint) <> cdouble) and (cstring1 >= 'ss'))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2139070 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1889990 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -873,7 +873,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterLongColLessLongColumn(col 1:smallint, col 0:smallint)(children: col 0:tinyint), FilterDoubleColGreaterEqualDoubleScalar(col 13:double, val 0.0)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double)), FilterDoubleColEqualDoubleScalar(col 4:float, val 17.0), FilterExprAndExpr(children: FilterTimestampColLessEqualTimestampColumn(col 9:timestamp, col 8:timestamp), FilterDoubleCo [...]
                     predicate: (((csmallint < UDFToShort(ctinyint)) and (UDFToDouble(ctimestamp1) >= 0.0D)) or (cfloat = 17.0) or ((ctimestamp2 <= ctimestamp1) and (UDFToDouble(cbigint) <> cdouble) and (cstring1 >= 'ss'))) (type: boolean)
-                    Statistics: Num rows: 2824 Data size: 491654 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2824 Data size: 434454 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), cbigint (type: bigint), cint (type: int), cfloat (type: float), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble(cint)) (type: double), UDFToDouble(cbigint) (type: double), (UDFToDouble(cbigint) * UDFToDouble(cbigint)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
@@ -882,7 +882,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [0, 3, 2, 4, 15, 18, 19, 22]
                           selectExpressions: CastLongToDouble(col 2:int) -> 15:double, DoubleColMultiplyDoubleColumn(col 16:double, col 17:double)(children: CastLongToDouble(col 2:int) -> 16:double, CastLongToDouble(col 2:int) -> 17:double) -> 18:double, CastLongToDouble(col 3:bigint) -> 19:double, DoubleColMultiplyDoubleColumn(col 20:double, col 21:double)(children: CastLongToDouble(col 3:bigint) -> 20:double, CastLongToDouble(col 3:bigint) -> 21:double) -> 22:double
-                      Statistics: Num rows: 2824 Data size: 491654 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2824 Data size: 434454 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0), count(_col0), max(_col1), sum(_col5), sum(_col4), count(_col2), sum(_col7), sum(_col6), count(_col1), max(_col3)
                         Group By Vectorization:
@@ -1111,7 +1111,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cboolean2 <> 1) and (CAST( csmallint AS decimal(8,3)) < 79.553) and (UDFToInteger(ctinyint) <> -257)) or ((cdouble > UDFToDouble(ctinyint)) and (cfloat >= UDFToFloat(cint))) or ((UDFToLong(cint) < cbigint) and (UDFToLong(ctinyint) > cbigint)) or (cstring1 regexp 'a.*' and (cstring2 like '%ss%'))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3056470 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2807390 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -1120,7 +1120,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterLongColNotEqualLongScalar(col 11:boolean, val 1), FilterDecimalColLessDecimalScalar(col 13:decimal(8,3), val 79.553)(children: CastLongToDecimal(col 1:smallint) -> 13:decimal(8,3)), FilterLongColNotEqualLongScalar(col 0:int, val -257)(children: col 0:tinyint)), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleColumn(col 5:double, col 14:double)(children: CastLongToDouble(col 0 [...]
                     predicate: (((cboolean2 <> 1) and (CAST( csmallint AS decimal(8,3)) < 79.553) and (UDFToInteger(ctinyint) <> -257)) or ((cdouble > UDFToDouble(ctinyint)) and (cfloat >= UDFToFloat(cint))) or ((UDFToLong(cint) < cbigint) and (UDFToLong(ctinyint) > cbigint)) or (cstring1 regexp 'a.*' and (cstring2 like '%ss%'))) (type: boolean)
-                    Statistics: Num rows: 9898 Data size: 2462086 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9898 Data size: 2261486 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int), cdouble (type: double), ctimestamp2 (type: timestamp), cstring1 (type: string), cboolean2 (type: boolean), ctinyint (type: tinyint), cfloat (type: float), ctimestamp1 (type: timestamp), csmallint (type: smallint), cbigint (type: bigint), (-3728L * cbigint) (type: bigint), (- cint) (type: int), (-863.257 - CAST( cint AS decimal(10,0))) (type: decimal(14,3)), (- csmallint) (type: smallint), (csmallint - (- csmallint)) (type: smallint), ( [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22
@@ -1129,7 +1129,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [2, 5, 9, 6, 11, 0, 4, 8, 1, 3, 16, 17, 19, 20, 22, 26, 29, 32, 33, 34, 36, 37, 39]
                           selectExpressions: LongScalarMultiplyLongColumn(val -3728, col 3:bigint) -> 16:bigint, LongColUnaryMinus(col 2:int) -> 17:int, DecimalScalarSubtractDecimalColumn(val -863.257, col 18:decimal(10,0))(children: CastLongToDecimal(col 2:int) -> 18:decimal(10,0)) -> 19:decimal(14,3), LongColUnaryMinus(col 1:smallint) -> 20:smallint, LongColSubtractLongColumn(col 1:smallint, col 21:smallint)(children: LongColUnaryMinus(col 1:smallint) -> 21:smallint) -> 22:smallint, Lo [...]
-                      Statistics: Num rows: 9898 Data size: 5632662 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9898 Data size: 5432062 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: double), _col2 (type: timestamp), _col3 (type: string), _col4 (type: boolean), _col5 (type: tinyint), _col6 (type: float), _col7 (type: timestamp), _col8 (type: smallint), _col9 (type: bigint), _col10 (type: bigint), _col11 (type: int), _col12 (type: decimal(14,3)), _col13 (type: smallint), _col14 (type: smallint), _col15 (type: smallint), _col16 (type: double), _col17 (type: decimal(15,3)), _col18 (type: float), _c [...]
                         sort order: +++++++++++++++++++++++
@@ -1137,7 +1137,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: 9898 Data size: 5632662 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9898 Data size: 5432062 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -1166,19 +1166,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22]
-                Statistics: Num rows: 9898 Data size: 5632662 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9898 Data size: 5432062 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 50
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 50 Data size: 28540 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 50 Data size: 27580 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 50 Data size: 28540 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 50 Data size: 27580 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1413,7 +1413,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cboolean1 < 0) or (cbigint = 359L) or ((UDFToInteger(ctinyint) < 197) and (UDFToLong(cint) = cbigint)) or ((cstring1 like '%ss') and (cfloat <= UDFToFloat(ctinyint)))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2477130 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -1422,7 +1422,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterLongColLessLongScalar(col 10:boolean, val 0), FilterLongColEqualLongScalar(col 3:bigint, val 359), FilterExprAndExpr(children: FilterLongColLessLongScalar(col 0:int, val 197)(children: col 0:tinyint), FilterLongColEqualLongColumn(col 2:bigint, col 3:bigint)(children: col 2:int)), FilterExprAndExpr(children: FilterStringColLikeStringScalar(col 6:string, pattern %ss), FilterDoubleColLessEqualDoubleColumn(col 4:fl [...]
                     predicate: ((cboolean1 < 0) or (cbigint = 359L) or ((UDFToInteger(ctinyint) < 197) and (UDFToLong(cint) = cbigint)) or ((cstring1 like '%ss') and (cfloat <= UDFToFloat(ctinyint)))) (type: boolean)
-                    Statistics: Num rows: 8194 Data size: 1734900 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 8194 Data size: 1651860 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int), cbigint (type: bigint), cstring1 (type: string), cboolean1 (type: boolean), cfloat (type: float), cdouble (type: double), ctimestamp2 (type: timestamp), csmallint (type: smallint), cstring2 (type: string), cboolean2 (type: boolean), (UDFToDouble(cint) / UDFToDouble(cbigint)) (type: double), (CAST( cbigint AS decimal(19,0)) % 79.553) (type: decimal(5,3)), (- (UDFToDouble(cint) / UDFToDouble(cbigint))) (type: double), (10.175 % cfloat) ( [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24
@@ -1431,7 +1431,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [2, 3, 6, 10, 4, 5, 9, 1, 7, 11, 16, 18, 22, 23, 24, 26, 29, 31, 32, 33, 37, 38, 39, 40, 41]
                           selectExpressions: DoubleColDivideDoubleColumn(col 14:double, col 15:double)(children: CastLongToDouble(col 2:int) -> 14:double, CastLongToDouble(col 3:bigint) -> 15:double) -> 16:double, DecimalColModuloDecimalScalar(col 17:decimal(19,0), val 79.553)(children: CastLongToDecimal(col 3:bigint) -> 17:decimal(19,0)) -> 18:decimal(5,3), DoubleColUnaryMinus(col 21:double)(children: DoubleColDivideDoubleColumn(col 19:double, col 20:double)(children: CastLongToDouble(c [...]
-                      Statistics: Num rows: 8194 Data size: 3349228 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 8194 Data size: 3266188 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: boolean), _col4 (type: float), _col5 (type: double), _col6 (type: timestamp), _col7 (type: smallint), _col8 (type: string), _col9 (type: boolean), _col10 (type: double), _col11 (type: decimal(5,3)), _col12 (type: double), _col13 (type: float), _col14 (type: float), _col15 (type: float), _col16 (type: float), _col17 (type: double), _col18 (type: double), _col19 (type: bigin [...]
                         sort order: +++++++++++++++++++++++++
@@ -1439,7 +1439,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: 8194 Data size: 3349228 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8194 Data size: 3266188 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -1468,19 +1468,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 21]
-                Statistics: Num rows: 8194 Data size: 3349228 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 8194 Data size: 3266188 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 25
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 25 Data size: 10520 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 25 Data size: 10320 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 25 Data size: 10520 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 25 Data size: 10320 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1664,7 +1664,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((UDFToFloat(cbigint) <= cfloat) and (CAST( csmallint AS decimal(7,2)) >= -26.28)) or ((cdouble <= UDFToDouble(cbigint)) and (cstring1 >= 'ss') and (UDFToDouble(cint) <> cdouble)) or (UDFToInteger(ctinyint) = -89010) or ((CAST( csmallint AS decimal(7,2)) > -26.28) and (cstring2 like 'ss'))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2477130 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -1673,7 +1673,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleColumn(col 13:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 3:bigint) -> 13:float), FilterDecimalColGreaterEqualDecimalScalar(col 14:decimal(7,2), val -26.28)(children: CastLongToDecimal(col 1:smallint) -> 14:decimal(7,2))), FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleColumn(col 5:double, col 15:double)(children: CastLongToDoub [...]
                     predicate: (((UDFToFloat(cbigint) <= cfloat) and (CAST( csmallint AS decimal(7,2)) >= -26.28)) or ((cdouble <= UDFToDouble(cbigint)) and (cstring1 >= 'ss') and (UDFToDouble(cint) <> cdouble)) or (UDFToInteger(ctinyint) = -89010) or ((CAST( csmallint AS decimal(7,2)) > -26.28) and (cstring2 like 'ss'))) (type: boolean)
-                    Statistics: Num rows: 10922 Data size: 2312410 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 10922 Data size: 2201730 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int), cstring1 (type: string), cboolean2 (type: boolean), ctimestamp2 (type: timestamp), cdouble (type: double), cfloat (type: float), cbigint (type: bigint), csmallint (type: smallint), cboolean1 (type: boolean), (cint + UDFToInteger(csmallint)) (type: int), (cbigint - UDFToLong(ctinyint)) (type: bigint), (- cbigint) (type: bigint), (- cfloat) (type: float), ((cbigint - UDFToLong(ctinyint)) + cbigint) (type: bigint), (cdouble / cdouble) (ty [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21
@@ -1682,7 +1682,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [2, 6, 11, 9, 5, 4, 3, 1, 10, 18, 19, 20, 21, 23, 24, 25, 28, 31, 33, 35, 36, 38]
                           selectExpressions: LongColAddLongColumn(col 2:int, col 1:int)(children: col 1:smallint) -> 18:int, LongColSubtractLongColumn(col 3:bigint, col 0:bigint)(children: col 0:tinyint) -> 19:bigint, LongColUnaryMinus(col 3:bigint) -> 20:bigint, DoubleColUnaryMinus(col 4:float) -> 21:float, LongColAddLongColumn(col 22:bigint, col 3:bigint)(children: LongColSubtractLongColumn(col 3:bigint, col 0:bigint)(children: col 0:tinyint) -> 22:bigint) -> 23:bigint, DoubleColDivide [...]
-                      Statistics: Num rows: 10922 Data size: 3594034 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 10922 Data size: 3483354 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col8 (type: boolean), _col1 (type: string), _col3 (type: timestamp), _col5 (type: float), _col6 (type: bigint), _col4 (type: double), _col0 (type: int), _col7 (type: smallint), _col9 (type: int), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: float), _col13 (type: bigint), _col14 (type: double), _col15 (type: double), _col16 (type: bigint), _col17 (type: double), _col18 (type: decimal(8,7)), _col19 (type: double), _col20 (type: small [...]
                         sort order: +++++++++++++++++++++
@@ -1690,7 +1690,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: 10922 Data size: 3594034 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 10922 Data size: 3483354 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col2 (type: boolean)
             Execution mode: vectorized, llap
@@ -1720,19 +1720,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [6, 1, 21, 2, 5, 3, 4, 7, 0, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20]
-                Statistics: Num rows: 10922 Data size: 3594034 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 10922 Data size: 3483354 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 75
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 75 Data size: 24810 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 75 Data size: 24090 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 75 Data size: 24810 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 75 Data size: 24090 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1973,7 +1973,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (((cdouble >= UDFToDouble(cfloat)) and (cstring2 <= 'a')) or ((CAST( cint AS decimal(13,3)) <= -1.389) and (csmallint < UDFToShort(ctinyint)) and (UDFToInteger(csmallint) < -6432)) or ((cstring1 like 'ss%') and (CAST( cbigint AS decimal(22,3)) < 10.175))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2528254 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2403694 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -1982,7 +1982,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColGreaterEqualDoubleColumn(col 5:double, col 4:double)(children: col 4:float), FilterStringGroupColLessEqualStringScalar(col 7:string, val a)), FilterExprAndExpr(children: FilterDecimalColLessEqualDecimalScalar(col 13:decimal(13,3), val -1.389)(children: CastLongToDecimal(col 2:int) -> 13:decimal(13,3)), FilterLongColLessLongColumn(col 1:smallint, col 0:smallint)(children: col [...]
                     predicate: (((cdouble >= UDFToDouble(cfloat)) and (cstring2 <= 'a')) or ((CAST( cint AS decimal(13,3)) <= -1.389) and (csmallint < UDFToShort(ctinyint)) and (UDFToInteger(csmallint) < -6432)) or ((cstring1 like 'ss%') and (CAST( cbigint AS decimal(22,3)) < 10.175))) (type: boolean)
-                    Statistics: Num rows: 3868 Data size: 795962 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3868 Data size: 756762 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctimestamp1 (type: timestamp), cstring2 (type: string), cdouble (type: double), cfloat (type: float), cbigint (type: bigint), csmallint (type: smallint), (UDFToDouble(cbigint) / 3569.0D) (type: double), (-257 - UDFToInteger(csmallint)) (type: int), (-6432.0 * cfloat) (type: float), (- cdouble) (type: double), (cdouble * 10.175D) (type: double), (UDFToDouble((-6432.0 * cfloat)) / UDFToDouble(cfloat)) (type: double), (- cfloat) (type: float), (cint % UDFT [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15
@@ -1991,7 +1991,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [8, 7, 5, 4, 3, 1, 16, 17, 18, 19, 20, 22, 23, 24, 25, 27]
                           selectExpressions: DoubleColDivideDoubleScalar(col 15:double, val 3569.0)(children: CastLongToDouble(col 3:bigint) -> 15:double) -> 16:double, LongScalarSubtractLongColumn(val -257, col 1:int)(children: col 1:smallint) -> 17:int, DoubleScalarMultiplyDoubleColumn(val -6432.0, col 4:float) -> 18:float, DoubleColUnaryMinus(col 5:double) -> 19:double, DoubleColMultiplyDoubleScalar(col 5:double, val 10.175) -> 20:double, DoubleColDivideDoubleColumn(col 21:double, col [...]
-                      Statistics: Num rows: 3868 Data size: 748844 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3868 Data size: 709644 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col5 (type: smallint), _col1 (type: string), _col2 (type: double), _col3 (type: float), _col4 (type: bigint), _col6 (type: double), _col7 (type: int), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: float), _col13 (type: int), _col14 (type: double), _col15 (type: double)
                         sort order: +++++++++++++++
@@ -1999,7 +1999,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: 3868 Data size: 748844 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3868 Data size: 709644 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: timestamp)
             Execution mode: vectorized, llap
@@ -2029,19 +2029,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [15, 1, 2, 3, 4, 0, 5, 6, 7, 8, 9, 10, 11, 12, 8, 14]
-                Statistics: Num rows: 3868 Data size: 748844 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3868 Data size: 709644 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 45
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 45 Data size: 8880 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 45 Data size: 8480 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 45 Data size: 8880 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 45 Data size: 8480 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2834,7 +2834,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((((UDFToInteger(ctinyint) <> -257) and cstring1 regexp '.*ss' and (UDFToDouble(ctimestamp1) > -3.0D) and cboolean2 is not null) or (UDFToDouble(ctimestamp2) = -5.0D) or ((UDFToDouble(ctimestamp1) < 0.0D) and (cstring2 like '%b%')) or (cdouble = UDFToDouble(cint)) or (cboolean1 is null and (cfloat < UDFToFloat(cint)))) and (UDFToDouble(ctimestamp1) <> 0.0D)) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 3019778 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2770698 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -2843,7 +2843,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterLongColNotEqualLongScalar(col 0:int, val -257)(children: col 0:tinyint), FilterStringColRegExpStringScalar(col 6:string, pattern .*ss), FilterDoubleColGreaterDoubleScalar(col 13:double, val -3.0)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), SelectColumnIsNotNull(col 11:boolean)), FilterDoubleColEqualDoubleScalar(col 14:double, val -5.0)( [...]
                     predicate: ((((UDFToInteger(ctinyint) <> -257) and cstring1 regexp '.*ss' and (UDFToDouble(ctimestamp1) > -3.0D) and cboolean2 is not null) or (UDFToDouble(ctimestamp2) = -5.0D) or ((UDFToDouble(ctimestamp1) < 0.0D) and (cstring2 like '%b%')) or (cdouble = UDFToDouble(cint)) or (cboolean1 is null and (cfloat < UDFToFloat(cint)))) and (UDFToDouble(ctimestamp1) <> 0.0D)) (type: boolean)
-                    Statistics: Num rows: 12288 Data size: 3019778 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12288 Data size: 2770698 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctimestamp1 (type: timestamp), cstring1 (type: string), cint (type: int), csmallint (type: smallint), ctinyint (type: tinyint), cfloat (type: float), cdouble (type: double), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble(cint)) (type: double), UDFToDouble(csmallint) (type: double), (UDFToDouble(csmallint) * UDFToDouble(csmallint)) (type: double), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: d [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -2852,7 +2852,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [8, 6, 2, 1, 0, 4, 5, 19, 22, 23, 26, 4, 27, 28, 31]
                           selectExpressions: CastLongToDouble(col 2:int) -> 19:double, DoubleColMultiplyDoubleColumn(col 20:double, col 21:double)(children: CastLongToDouble(col 2:int) -> 20:double, CastLongToDouble(col 2:int) -> 21:double) -> 22:double, CastLongToDouble(col 1:smallint) -> 23:double, DoubleColMultiplyDoubleColumn(col 24:double, col 25:double)(children: CastLongToDouble(col 1:smallint) -> 24:double, CastLongToDouble(col 1:smallint) -> 25:double) -> 26:double, DoubleColMul [...]
-                      Statistics: Num rows: 12288 Data size: 3019778 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 12288 Data size: 2770698 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col8), sum(_col7), count(_col2), sum(_col3), count(_col3), count(), min(_col4), sum(_col10), sum(_col9), sum(_col12), sum(_col11), count(_col5), sum(_col2), sum(_col5), min(_col6), sum(_col14), sum(_col13), count(_col4)
                         Group By Vectorization:
@@ -2864,10 +2864,10 @@ STAGE PLANS:
                             vectorProcessingMode: HASH
                             projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17]
                         keys: _col0 (type: timestamp), _col1 (type: string)
-                        minReductionHashAggr: 0.99
+                        minReductionHashAggr: 0.5133463
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19
-                        Statistics: Num rows: 6144 Data size: 1537192 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 5980 Data size: 1435604 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: timestamp), _col1 (type: string)
                           sort order: ++
@@ -2876,7 +2876,7 @@ STAGE PLANS:
                               className: VectorReduceSinkMultiKeyOperator
                               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: 6144 Data size: 1537192 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 5980 Data size: 1435604 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col2 (type: double), _col3 (type: double), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: tinyint), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: double), _col13 (type: bigint), _col14 (type: bigint), _col15 (type: double), _col16 (type: double), _col17 (type: double), _col18 (type: double), _col19 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -2911,7 +2911,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: timestamp), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19
-                Statistics: Num rows: 3072 Data size: 768596 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5980 Data size: 1435604 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: timestamp), _col1 (type: string), power(((_col2 - ((_col3 * _col3) / _col4)) / _col4), 0.5) (type: double), (power(((_col2 - ((_col3 * _col3) / _col4)) / _col4), 0.5) * 10.175D) (type: double), (- power(((_col2 - ((_col3 * _col3) / _col4)) / _col4), 0.5)) (type: double), (UDFToDouble(_col5) / _col6) (type: double), (- power(((_col2 - ((_col3 * _col3) / _col4)) / _col4), 0.5)) (type: double), (-26.28D - power(((_col2 - ((_col3 * _col3) / _col4)) [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38
@@ -2920,7 +2920,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 1, 24, 30, 36, 38, 44, 50, 7, 51, 64, 8, 80, 87, 94, 118, 125, 133, 137, 140, 142, 149, 190, 198, 199, 241, 245, 246, 16, 250, 264, 272, 275, 280, 14, 309, 311, 7, 336]
                       selectExpressions: FuncPowerDoubleToDouble(col 23:double)(children: DoubleColDivideLongColumn(col 22:double, col 4:bigint)(children: DoubleColSubtractDoubleColumn(col 2:double, col 21:double)(children: DoubleColDivideLongColumn(col 20:double, col 4:bigint)(children: DoubleColMultiplyDoubleColumn(col 3:double, col 3:double) -> 20:double) -> 21:double) -> 22:double) -> 23:double) -> 24:double, DoubleColMultiplyDoubleScalar(col 29:double, val 10.175)(children: FuncPowe [...]
-                  Statistics: Num rows: 3072 Data size: 1542740 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5980 Data size: 2942564 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: timestamp), _col1 (type: string), _col2 (type: double), _col3 (type: double), _col4 (type: double), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: bigint), _col9 (type: bigint), _col10 (type: double), _col11 (type: tinyint), _col12 (type: double), _col13 (type: double), _col14 (type: double), _col15 (type: double), _col16 (type: double), _col17 (type: double), _col18 (type: double), _col19 (type: double), _ [...]
                     sort order: +++++++++++++++++++++++++++++++++++++++
@@ -2928,7 +2928,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: 3072 Data size: 1542740 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 5980 Data size: 2942564 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.1
         Reducer 3 
             Execution mode: vectorized, llap
@@ -2946,19 +2946,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 4, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 8, 38]
-                Statistics: Num rows: 3072 Data size: 1542740 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5980 Data size: 2942564 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 50
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 50 Data size: 25172 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 50 Data size: 24692 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 50 Data size: 25172 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 50 Data size: 24692 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3242,7 +3242,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: (cboolean1 is not null and (((cdouble < UDFToDouble(csmallint)) and (cboolean2 = cboolean1) and (CAST( cbigint AS decimal(22,3)) <= -863.257)) or ((cint >= -257) and (cboolean1 >= 1) and cstring1 is not null) or cstring2 regexp 'b' or ((csmallint >= UDFToShort(ctinyint)) and ctimestamp2 is null))) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2477130 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -3251,7 +3251,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 10:boolean), FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleColumn(col 5:double, col 13:double)(children: CastLongToDouble(col 1:smallint) -> 13:double), FilterLongColEqualLongColumn(col 11:boolean, col 10:boolean), FilterDecimalColLessEqualDecimalScalar(col 14:decimal(22,3), val -863.257)(children: CastLongToDecimal(col 3:bigint) -> 14:decimal(22,3))), Filt [...]
                     predicate: (cboolean1 is not null and (((cdouble < UDFToDouble(csmallint)) and (cboolean2 = cboolean1) and (CAST( cbigint AS decimal(22,3)) <= -863.257)) or ((cint >= -257) and (cboolean1 >= 1) and cstring1 is not null) or cstring2 regexp 'b' or ((csmallint >= UDFToShort(ctinyint)) and ctimestamp2 is null))) (type: boolean)
-                    Statistics: Num rows: 6237 Data size: 1320590 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7012 Data size: 1413600 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cboolean1 (type: boolean), cfloat (type: float), cbigint (type: bigint), cint (type: int), cdouble (type: double), ctinyint (type: tinyint), csmallint (type: smallint), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble(cint)) (type: double), UDFToDouble(cbigint) (type: double), (UDFToDouble(cbigint) * UDFToDouble(cbigint)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -3260,7 +3260,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [10, 4, 3, 2, 5, 0, 1, 15, 18, 19, 22, 23, 26, 27, 30]
                           selectExpressions: CastLongToDouble(col 2:int) -> 15:double, DoubleColMultiplyDoubleColumn(col 16:double, col 17:double)(children: CastLongToDouble(col 2:int) -> 16:double, CastLongToDouble(col 2:int) -> 17:double) -> 18:double, CastLongToDouble(col 3:bigint) -> 19:double, DoubleColMultiplyDoubleColumn(col 20:double, col 21:double)(children: CastLongToDouble(col 3:bigint) -> 20:double, CastLongToDouble(col 3:bigint) -> 21:double) -> 22:double, CastLongToDouble(c [...]
-                      Statistics: Num rows: 6237 Data size: 1320590 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7012 Data size: 1413600 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: max(_col1), sum(_col2), sum(_col8), sum(_col7), count(_col3), sum(_col4), count(_col4), min(_col2), sum(_col10), sum(_col9), count(_col2), sum(_col3), sum(_col12), sum(_col11), count(_col5), sum(_col14), sum(_col13), count(_col6)
                         Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out b/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
index 333cbd3..6a3c158 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
@@ -170,7 +170,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cbigint % 250L) = 0L) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 1684250 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1559690 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -180,7 +180,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColEqualLongScalar(col 13:bigint, val 0)(children: LongColModuloLongScalar(col 3:bigint, val 250) -> 13:bigint)
                     predicate: ((cbigint % 250L) = 0L) (type: boolean)
-                    Statistics: Num rows: 6144 Data size: 842180 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6144 Data size: 779900 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: UDFToBoolean(ctinyint) (type: boolean), UDFToBoolean(csmallint) (type: boolean), UDFToBoolean(cint) (type: boolean), UDFToBoolean(cbigint) (type: boolean), UDFToBoolean(cfloat) (type: boolean), UDFToBoolean(cdouble) (type: boolean), cboolean1 (type: boolean), ((cbigint * 0L) <> 0L) (type: boolean), UDFToBoolean(ctimestamp1) (type: boolean), UDFToBoolean(cstring1) (type: boolean), UDFToInteger(ctinyint) (type: int), UDFToInteger(csmallint) (type: int), c [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50, _col51, _col52, _col53, _col54, _col55, _col56, _col5 [...]
@@ -189,13 +189,13 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [14, 15, 16, 17, 18, 19, 10, 21, 22, 23, 0, 1, 2, 3, 24, 25, 10, 26, 27, 29, 30, 31, 32, 33, 34, 35, 36, 4, 5, 37, 38, 39, 41, 42, 5, 44, 46, 48, 50, 51, 52, 54, 58, 60, 8, 61, 63, 64, 65, 66, 67, 68, 69, 70, 72, 73, 6, 74, 75, 77, 79, 81, 84]
                           selectExpressions: CastLongToBooleanViaLongToLong(col 0:tinyint) -> 14:boolean, CastLongToBooleanViaLongToLong(col 1:smallint) -> 15:boolean, CastLongToBooleanViaLongToLong(col 2:int) -> 16:boolean, CastLongToBooleanViaLongToLong(col 3:bigint) -> 17:boolean, CastDoubleToBooleanViaDoubleToLong(col 4:float) -> 18:boolean, CastDoubleToBooleanViaDoubleToLong(col 5:double) -> 19:boolean, LongColNotEqualLongScalar(col 20:bigint, val 0)(children: LongColMultiplyLongSca [...]
-                      Statistics: Num rows: 6144 Data size: 16362860 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6144 Data size: 16014092 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
                         File Sink Vectorization:
                             className: VectorFileSinkOperator
                             native: false
-                        Statistics: Num rows: 6144 Data size: 16362860 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6144 Data size: 16014092 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 d455e35..ea8a7fa 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
@@ -263,7 +263,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 52 Data size: 4276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 3956 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -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: 16756 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
                       sort order: +
@@ -282,7 +282,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: 16756 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 52 Data size: 16436 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
@@ -311,13 +311,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: 16756 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 52 Data size: 16756 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -675,7 +675,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 52 Data size: 7617 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 7497 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -1010,7 +1010,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -1020,7 +1020,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [1]
-                    Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: min(ctimestamp1), max(ctimestamp1), count(ctimestamp1), count()
                       Group By Vectorization:
@@ -1142,7 +1142,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -1152,7 +1152,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [1]
-                    Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: sum(ctimestamp1)
                       Group By Vectorization:
@@ -1291,7 +1291,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -1302,7 +1302,7 @@ STAGE PLANS:
                         native: true
                         projectedOutputColumnNums: [1, 5, 8]
                         selectExpressions: CastTimestampToDouble(col 1:timestamp) -> 5:double, DoubleColMultiplyDoubleColumn(col 6:double, col 7:double)(children: CastTimestampToDouble(col 1:timestamp) -> 6:double, CastTimestampToDouble(col 1:timestamp) -> 7:double) -> 8:double
-                    Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: sum(_col0), count(_col0), sum(_col2), sum(_col1)
                       Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_ints_casts.q.out b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_ints_casts.q.out
index e13d4a9..0bc7387 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_ints_casts.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_ints_casts.q.out
@@ -56,7 +56,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cbigint % 250L) = 0L) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 1684250 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1559690 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -65,7 +65,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColEqualLongScalar(col 13:bigint, val 0)(children: LongColModuloLongScalar(col 3:bigint, val 250) -> 13:bigint)
                     predicate: ((cbigint % 250L) = 0L) (type: boolean)
-                    Statistics: Num rows: 6144 Data size: 842180 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6144 Data size: 779900 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0L) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timest [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -74,13 +74,13 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [15, 17, 19, 21, 22, 23, 25, 29, 8, 30, 32]
                           selectExpressions: CastMillisecondsLongToTimestamp(col 0:tinyint) -> 15:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 17:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 19:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 21:timestamp, CastDoubleToTimestamp(col 4:float) -> 22:timestamp, CastDoubleToTimestamp(col 5:double) -> 23:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 25:timestamp, CastMillisecondsLongToTim [...]
-                      Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6144 Data size: 2641080 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
                         File Sink Vectorization:
                             className: VectorFileSinkOperator
                             native: false
-                        Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6144 Data size: 2641080 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -225,7 +225,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: ((cbigint % 250L) = 0L) (type: boolean)
-                  Statistics: Num rows: 12288 Data size: 1684250 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 1559690 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Filter Operator
@@ -234,7 +234,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColEqualLongScalar(col 13:bigint, val 0)(children: LongColModuloLongScalar(col 3:bigint, val 250) -> 13:bigint)
                     predicate: ((cbigint % 250L) = 0L) (type: boolean)
-                    Statistics: Num rows: 6144 Data size: 842180 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6144 Data size: 779900 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0L) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timest [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -243,13 +243,13 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [14, 15, 16, 17, 18, 19, 20, 22, 8, 23, 25]
                           selectExpressions: CastLongToTimestamp(col 0:tinyint) -> 14:timestamp, CastLongToTimestamp(col 1:smallint) -> 15:timestamp, CastLongToTimestamp(col 2:int) -> 16:timestamp, CastLongToTimestamp(col 3:bigint) -> 17:timestamp, CastDoubleToTimestamp(col 4:float) -> 18:timestamp, CastDoubleToTimestamp(col 5:double) -> 19:timestamp, CastLongToTimestamp(col 10:boolean) -> 20:timestamp, CastLongToTimestamp(col 21:bigint)(children: LongColMultiplyLongScalar(col 3:bigint,  [...]
-                      Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6144 Data size: 2641080 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
                         File Sink Vectorization:
                             className: VectorFileSinkOperator
                             native: false
-                        Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6144 Data size: 2641080 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/orc_merge5.q.out b/ql/src/test/results/clientpositive/orc_merge5.q.out
index df5651e..da18e7c 100644
--- a/ql/src/test/results/clientpositive/orc_merge5.q.out
+++ b/ql/src/test/results/clientpositive/orc_merge5.q.out
@@ -67,20 +67,20 @@ STAGE PLANS:
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                    Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       sort order: 
-                      Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: struct< [...]
+                      Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: struct< [...]
       Reduce Operator Tree:
         Group By Operator
           aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2), compute_stats(VALUE._col3), compute_stats(VALUE._col4)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -188,20 +188,20 @@ STAGE PLANS:
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                    Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       sort order: 
-                      Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: struct< [...]
+                      Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: struct< [...]
       Reduce Operator Tree:
         Group By Operator
           aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2), compute_stats(VALUE._col3), compute_stats(VALUE._col4)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
             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/orc_merge6.q.out b/ql/src/test/results/clientpositive/orc_merge6.q.out
index bc05d2f..4a8f975 100644
--- a/ql/src/test/results/clientpositive/orc_merge6.q.out
+++ b/ql/src/test/results/clientpositive/orc_merge6.q.out
@@ -74,7 +74,7 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
                       Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col6 (type: struct< [...]
+                      value expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col6 (type: struct< [...]
       Reduce Operator Tree:
         Group By Operator
           aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2), compute_stats(VALUE._col3), compute_stats(VALUE._col4)
@@ -83,7 +83,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
           Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),cou [...]
+            expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),cou [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
             Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
@@ -248,7 +248,7 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
                       Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col6 (type: struct< [...]
+                      value expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col6 (type: struct< [...]
       Reduce Operator Tree:
         Group By Operator
           aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2), compute_stats(VALUE._col3), compute_stats(VALUE._col4)
@@ -257,7 +257,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
           Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),cou [...]
+            expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col5 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),cou [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
             Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
diff --git a/ql/src/test/results/clientpositive/orc_merge_incompat1.q.out b/ql/src/test/results/clientpositive/orc_merge_incompat1.q.out
index 538e2a0..bd59c4a 100644
--- a/ql/src/test/results/clientpositive/orc_merge_incompat1.q.out
+++ b/ql/src/test/results/clientpositive/orc_merge_incompat1.q.out
@@ -67,20 +67,20 @@ STAGE PLANS:
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                    Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       sort order: 
-                      Statistics: Num rows: 1 Data size: 2696 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: struct< [...]
+                      Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col4 (type: struct< [...]
       Reduce Operator Tree:
         Group By Operator
           aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2), compute_stats(VALUE._col3), compute_stats(VALUE._col4)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 2760 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 1 Data size: 2824 Basic stats: COMPLETE Column stats: NONE
             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/orc_merge_incompat2.q.out b/ql/src/test/results/clientpositive/orc_merge_incompat2.q.out
index b3cbf4a..58e2654 100644
--- a/ql/src/test/results/clientpositive/orc_merge_incompat2.q.out
+++ b/ql/src/test/results/clientpositive/orc_merge_incompat2.q.out
@@ -62,7 +62,7 @@ STAGE PLANS:
                     sort order: +
                     Map-reduce partition columns: _col0 (type: double)
                     Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col5 (type: struct<co [...]
+                    value expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),countnulls:bigint,bitvector:binary>), _col5 (type: struct<co [...]
               File Output Operator
                 compressed: false
                 table:
@@ -77,7 +77,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
           Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),cou [...]
+            expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col3 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col4 (type: struct<columntype:string,min:decimal(38,0),max:decimal(38,0),cou [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
             Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
diff --git a/ql/src/test/results/clientpositive/parquet_vectorization_0.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_0.q.out
index 7d82465..158157a 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_0.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_0.q.out
@@ -29912,7 +29912,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (((cint = 49) and (cfloat = 3.5)) or ((cint = 47) and (cfloat = 2.09)) or ((cint = 45) and (cfloat = 3.02))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
@@ -30033,7 +30033,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (struct(cint,cfloat)) IN (const struct(49,3.5), const struct(47,2.09), const struct(45,3.02)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
@@ -30153,7 +30153,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (((cint = 49) or (cfloat = 3.5)) and ((cint = 47) or (cfloat = 2.09)) and ((cint = 45) or (cfloat = 3.02))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
diff --git a/ql/src/test/results/clientpositive/parquet_vectorization_10.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_10.q.out
index 0cc7e67..776f7c8 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_10.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_10.q.out
@@ -65,7 +65,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: ((cstring2 <= '10') or ((UDFToDouble(ctinyint) > cdouble) and (CAST( ctinyint AS decimal(6,2)) <= -5638.15)) or ((cdouble > 6981.0D) and ((CAST( csmallint AS decimal(11,4)) = 9763215.5639) or (cstring1 like '%a')))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -74,7 +74,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterStringGroupColLessEqualStringScalar(col 7:string, val 10), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleColumn(col 13:double, col 5:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterDecimalColLessEqualDecimalScalar(col 14:decimal(6,2), val -5638.15)(children: CastLongToDecimal(col 0:tinyint) -> 14:decimal(6,2))), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 5:doub [...]
               predicate: ((cstring2 <= '10') or ((UDFToDouble(ctinyint) > cdouble) and (CAST( ctinyint AS decimal(6,2)) <= -5638.15)) or ((cdouble > 6981.0D) and ((CAST( csmallint AS decimal(11,4)) = 9763215.5639) or (cstring1 like '%a')))) (type: boolean)
-              Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cdouble (type: double), ctimestamp1 (type: timestamp), ctinyint (type: tinyint), cboolean1 (type: boolean), cstring1 (type: string), (- cdouble) (type: double), (cdouble + UDFToDouble(csmallint)) (type: double), ((cdouble + UDFToDouble(csmallint)) % 33.0D) (type: double), (- cdouble) (type: double), (UDFToDouble(ctinyint) % cdouble) (type: double), (UDFToShort(ctinyint) % csmallint) (type: smallint), (- cdouble) (type: double), (cbigint * UDFToLong((UDFToShor [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -83,13 +83,13 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [5, 8, 0, 10, 6, 16, 18, 21, 22, 24, 25, 26, 28, 31, 33]
                     selectExpressions: DoubleColUnaryMinus(col 5:double) -> 16:double, DoubleColAddDoubleColumn(col 5:double, col 17:double)(children: CastLongToDouble(col 1:smallint) -> 17:double) -> 18:double, DoubleColModuloDoubleScalar(col 20:double, val 33.0)(children: DoubleColAddDoubleColumn(col 5:double, col 19:double)(children: CastLongToDouble(col 1:smallint) -> 19:double) -> 20:double) -> 21:double, DoubleColUnaryMinus(col 5:double) -> 22:double, DoubleColModuloDoubleColumn(co [...]
-                Statistics: Num rows: 12288 Data size: 2434654 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12288 Data size: 2310094 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 12288 Data size: 2434654 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2310094 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/parquet_vectorization_11.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_11.q.out
index 97f5ede..5603015 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_11.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_11.q.out
@@ -47,7 +47,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: ((ctimestamp1 is null and (cstring1 like '%a')) or (cstring2 = cstring1)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2381474 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2256914 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -56,7 +56,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: SelectColumnIsNull(col 8:timestamp), FilterStringColLikeStringScalar(col 6:string, pattern %a)), FilterStringGroupColEqualStringGroupColumn(col 7:string, col 6:string))
               predicate: ((ctimestamp1 is null and (cstring1 like '%a')) or (cstring2 = cstring1)) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 1190792 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 7701 Data size: 1414500 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cstring1 (type: string), cboolean1 (type: boolean), cdouble (type: double), ctimestamp1 (type: timestamp), (-3728 * UDFToInteger(csmallint)) (type: int), (cdouble - 9763215.5639D) (type: double), (- cdouble) (type: double), ((- cdouble) + 6981.0D) (type: double), (cdouble * -5638.15D) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -65,13 +65,13 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [6, 10, 5, 8, 13, 14, 15, 17, 18]
                     selectExpressions: LongScalarMultiplyLongColumn(val -3728, col 1:int)(children: col 1:smallint) -> 13:int, DoubleColSubtractDoubleScalar(col 5:double, val 9763215.5639) -> 14:double, DoubleColUnaryMinus(col 5:double) -> 15:double, DoubleColAddDoubleScalar(col 16:double, val 6981.0)(children: DoubleColUnaryMinus(col 5:double) -> 16:double) -> 17:double, DoubleColMultiplyDoubleScalar(col 5:double, val -5638.15) -> 18:double
-                Statistics: Num rows: 6144 Data size: 953272 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7701 Data size: 1116736 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 6144 Data size: 953272 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 7701 Data size: 1116736 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/parquet_vectorization_12.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_12.q.out
index 6059ac9..b60816e 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_12.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_12.q.out
@@ -82,7 +82,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (((cstring1 like '%a') or ((cboolean2 <= 1) and (cbigint >= UDFToLong(csmallint)))) and ((cboolean1 >= cboolean2) or (UDFToShort(ctinyint) <> csmallint)) and ctimestamp1 is null) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1647554 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1522994 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -91,7 +91,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterStringColLikeStringScalar(col 6:string, pattern %a), FilterExprAndExpr(children: FilterLongColLessEqualLongScalar(col 11:boolean, val 1), FilterLongColGreaterEqualLongColumn(col 3:bigint, col 1:bigint)(children: col 1:smallint))), FilterExprOrExpr(children: FilterLongColGreaterEqualLongColumn(col 10:boolean, col 11:boolean), FilterLongColNotEqualLongColumn(col 0:smallint, col 1:smallint)(c [...]
               predicate: (((cstring1 like '%a') or ((cboolean2 <= 1) and (cbigint >= UDFToLong(csmallint)))) and ((cboolean1 >= cboolean2) or (UDFToShort(ctinyint) <> csmallint)) and ctimestamp1 is null) (type: boolean)
-              Statistics: Num rows: 1 Data size: 166 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1903 Data size: 236052 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cbigint (type: bigint), cboolean1 (type: boolean), cstring1 (type: string), cdouble (type: double), UDFToDouble(cbigint) (type: double), (UDFToDouble(cbigint) * UDFToDouble(cbigint)) (type: double), (cdouble * cdouble) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
@@ -100,7 +100,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [3, 10, 6, 5, 13, 16, 17]
                     selectExpressions: CastLongToDouble(col 3:bigint) -> 13:double, DoubleColMultiplyDoubleColumn(col 14:double, col 15:double)(children: CastLongToDouble(col 3:bigint) -> 14:double, CastLongToDouble(col 3:bigint) -> 15:double) -> 16:double, DoubleColMultiplyDoubleColumn(col 5:double, col 5:double) -> 17:double
-                Statistics: Num rows: 1 Data size: 166 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1903 Data size: 236052 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(_col0), sum(_col5), sum(_col4), sum(_col3), count(_col3), sum(_col0), sum(_col6)
                   Group By Vectorization:
@@ -115,7 +115,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-                  Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1903 Data size: 268676 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: double), _col1 (type: bigint), _col2 (type: string), _col3 (type: boolean)
                     sort order: ++++
@@ -125,7 +125,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: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1903 Data size: 268676 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col4 (type: bigint), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: bigint), _col9 (type: bigint), _col10 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -147,11 +147,11 @@ STAGE PLANS:
           keys: KEY._col0 (type: double), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: boolean)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-          Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1903 Data size: 268676 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col1 (type: bigint), _col3 (type: boolean), _col2 (type: string), _col0 (type: double), (-6432.0D * _col0) (type: double), (- _col1) (type: bigint), _col4 (type: bigint), (_col1 * _col4) (type: bigint), power(((_col5 - ((_col6 * _col6) / _col4)) / CASE WHEN ((_col4 = 1L)) THEN (null) ELSE ((_col4 - 1)) END), 0.5) (type: double), ((-6432.0D * _col0) / -6432.0D) (type: double), (- ((-6432.0D * _col0) / -6432.0D)) (type: double), (_col7 / _col8) (type: double), (-  [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col17, _col18, _col19
-            Statistics: Num rows: 1 Data size: 346 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1903 Data size: 603604 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -173,7 +173,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: 1 Data size: 346 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1903 Data size: 603604 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col1 (type: boolean), _col4 (type: double), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: double), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: double), _col13 (type: decimal(22,2)), _col14 (type: bigint), _col15 (type: double), _col17 (type: double), _col18 (type: double), _col19 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -193,10 +193,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey1 (type: bigint), VALUE._col0 (type: boolean), KEY.reducesinkkey2 (type: string), null (type: timestamp), KEY.reducesinkkey0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: bigint), VALUE._col3 (type: bigint), VALUE._col4 (type: bigint), VALUE._col5 (type: double), VALUE._col6 (type: double), VALUE._col7 (type: double), VALUE._col8 (type: double), VALUE._col9 (type: double), VALUE._col10 (type: decimal(22,2)), VALUE._col11 (type: big [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-          Statistics: Num rows: 1 Data size: 386 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1903 Data size: 603644 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 386 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1903 Data size: 603644 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/parquet_vectorization_13.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_13.q.out
index 134ce1f..90462a6 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_13.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_13.q.out
@@ -84,7 +84,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28789.0D) and (UDFToDouble(ctimestamp2) <> -28788.0D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2028982 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1779902 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -93,7 +93,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val 3569.0), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 10.175), FilterLongColNotEqualLongScalar(col 10:boolean, val 1)), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28789.0)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val -28788.0 [...]
               predicate: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28789.0D) and (UDFToDouble(ctimestamp2) <> -28788.0D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-              Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -102,7 +102,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [10, 0, 8, 4, 6, 4, 16, 17, 20]
                     selectExpressions: DoubleColMultiplyDoubleColumn(col 4:double, col 4:double)(children: col 4:float, col 4:float) -> 16:double, CastLongToDouble(col 0:tinyint) -> 17:double, DoubleColMultiplyDoubleColumn(col 18:double, col 19:double)(children: CastLongToDouble(col 0:tinyint) -> 18:double, CastLongToDouble(col 0:tinyint) -> 19:double) -> 20:double
-                Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: max(_col1), sum(_col3), sum(_col6), sum(_col5), count(_col3), sum(_col8), sum(_col7), count(_col1), max(_col3), min(_col1)
                   Group By Vectorization:
@@ -117,7 +117,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-                  Statistics: Num rows: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
                     sort order: +++++
@@ -127,7 +127,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: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: double), _col8 (type: double), _col9 (type: bigint), _col10 (type: double), _col11 (type: double), _col12 (type: bigint), _col13 (type: float), _col14 (type: tinyint)
       Execution mode: vectorized
       Map Vectorization:
@@ -149,11 +149,11 @@ STAGE PLANS:
           keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-          Statistics: Num rows: 346 Data size: 64822 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * _col3) (type: float), power(((_col7 - ((_col8 * _col8) / _col9)) / _col9), 0.5) (type: double), (- _col6) (type: double), power(((_col10 - ((_col11 * [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-            Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -175,7 +175,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: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
               TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -195,13 +195,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: boolean), KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey2 (type: timestamp), KEY.reducesinkkey3 (type: float), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: tinyint), KEY.reducesinkkey6 (type: tinyint), KEY.reducesinkkey7 (type: tinyint), KEY.reducesinkkey8 (type: double), KEY.reducesinkkey9 (type: double), KEY.reducesinkkey10 (type: double), KEY.reducesinkkey11 (type: float), KEY.reducesinkkey12 (type: double), KEY.r [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-          Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 40
-            Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 40 Data size: 12846 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 40 Data size: 12846 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -407,7 +407,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28801.388D) and (UDFToDouble(ctimestamp2) <> -28801.336D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2028982 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1779902 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -416,7 +416,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val 3569.0), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 10.175), FilterLongColNotEqualLongScalar(col 10:boolean, val 1)), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28801.388)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val -28801 [...]
               predicate: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28801.388D) and (UDFToDouble(ctimestamp2) <> -28801.336D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-              Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -425,7 +425,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [10, 0, 8, 4, 6, 4, 16, 17, 20]
                     selectExpressions: DoubleColMultiplyDoubleColumn(col 4:double, col 4:double)(children: col 4:float, col 4:float) -> 16:double, CastLongToDouble(col 0:tinyint) -> 17:double, DoubleColMultiplyDoubleColumn(col 18:double, col 19:double)(children: CastLongToDouble(col 0:tinyint) -> 18:double, CastLongToDouble(col 0:tinyint) -> 19:double) -> 20:double
-                Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: max(_col1), sum(_col3), sum(_col6), sum(_col5), count(_col3), sum(_col8), sum(_col7), count(_col1), max(_col3), min(_col1)
                   Group By Vectorization:
@@ -440,7 +440,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-                  Statistics: Num rows: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
                     sort order: +++++
@@ -450,7 +450,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: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: double), _col8 (type: double), _col9 (type: bigint), _col10 (type: double), _col11 (type: double), _col12 (type: bigint), _col13 (type: float), _col14 (type: tinyint)
       Execution mode: vectorized
       Map Vectorization:
@@ -472,11 +472,11 @@ STAGE PLANS:
           keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-          Statistics: Num rows: 346 Data size: 64822 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * _col3) (type: float), power(((_col7 - ((_col8 * _col8) / _col9)) / _col9), 0.5) (type: double), (- _col6) (type: double), power(((_col10 - ((_col11 * [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-            Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -498,7 +498,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: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
               TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -518,13 +518,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: boolean), KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey2 (type: timestamp), KEY.reducesinkkey3 (type: float), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: tinyint), KEY.reducesinkkey6 (type: tinyint), KEY.reducesinkkey7 (type: tinyint), KEY.reducesinkkey8 (type: double), KEY.reducesinkkey9 (type: double), KEY.reducesinkkey10 (type: double), KEY.reducesinkkey11 (type: float), KEY.reducesinkkey12 (type: double), KEY.r [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-          Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 40
-            Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 40 Data size: 12846 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 40 Data size: 12846 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/parquet_vectorization_14.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_14.q.out
index c943d7b..26948e3 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_14.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_14.q.out
@@ -84,7 +84,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: ((UDFToLong(ctinyint) <= cbigint) and (cdouble < UDFToDouble(ctinyint)) and ((cbigint > -257L) or (cfloat < UDFToFloat(cint))) and ((UDFToDouble(cint) <= cdouble) or (ctimestamp2 < ctimestamp1))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2139070 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1889990 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -93,7 +93,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterLongColLessEqualLongColumn(col 0:bigint, col 3:bigint)(children: col 0:tinyint), FilterDoubleColLessDoubleColumn(col 5:double, col 13:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterExprOrExpr(children: FilterLongColGreaterLongScalar(col 3:bigint, val -257), FilterDoubleColLessDoubleColumn(col 4:float, col 14:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 14:float)), FilterExprO [...]
               predicate: ((UDFToLong(ctinyint) <= cbigint) and (cdouble < UDFToDouble(ctinyint)) and ((cbigint > -257L) or (cfloat < UDFToFloat(cint))) and ((UDFToDouble(cint) <= cdouble) or (ctimestamp2 < ctimestamp1))) (type: boolean)
-              Statistics: Num rows: 758 Data size: 132082 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 758 Data size: 116802 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string), cboolean1 (type: boolean), cdouble (type: double), (- (-26.28D + cdouble)) (type: double), ((- (-26.28D + cdouble)) * (- (-26.28D + cdouble))) (type: double), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -102,7 +102,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [8, 4, 6, 10, 5, 17, 22, 4, 23]
                     selectExpressions: DoubleColUnaryMinus(col 16:double)(children: DoubleScalarAddDoubleColumn(val -26.28, col 5:double) -> 16:double) -> 17:double, DoubleColMultiplyDoubleColumn(col 19:double, col 21:double)(children: DoubleColUnaryMinus(col 18:double)(children: DoubleScalarAddDoubleColumn(val -26.28, col 5:double) -> 18:double) -> 19:double, DoubleColUnaryMinus(col 20:double)(children: DoubleScalarAddDoubleColumn(val -26.28, col 5:double) -> 20:double) -> 21:double) -> [...]
-                Statistics: Num rows: 758 Data size: 132082 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 758 Data size: 116802 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col6), sum(_col5), count(_col5), max(_col1), sum(_col8), sum(_col7), count(_col1)
                   Group By Vectorization:
@@ -117,7 +117,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                  Statistics: Num rows: 379 Data size: 66108 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 379 Data size: 62308 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: float), _col2 (type: double), _col3 (type: timestamp), _col4 (type: boolean)
                     sort order: +++++
@@ -127,7 +127,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: 379 Data size: 66108 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 379 Data size: 62308 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col5 (type: double), _col6 (type: double), _col7 (type: bigint), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: bigint)
       Execution mode: vectorized
       Map Vectorization:
@@ -149,11 +149,11 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: float), KEY._col2 (type: double), KEY._col3 (type: timestamp), KEY._col4 (type: boolean)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-          Statistics: Num rows: 189 Data size: 33008 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 379 Data size: 62308 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col3 (type: timestamp), _col1 (type: float), _col0 (type: string), _col4 (type: boolean), _col2 (type: double), (-26.28D + _col2) (type: double), (- (-26.28D + _col2)) (type: double), power(((_col5 - ((_col6 * _col6) / _col7)) / CASE WHEN ((_col7 = 1L)) THEN (null) ELSE ((_col7 - 1)) END), 0.5) (type: double), (_col1 * -26.28) (type: float), _col8 (type: float), (- _col1) (type: float), (- _col8) (type: float), ((- (-26.28D + _col2)) / 10.175D) (type: double), p [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21
-            Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 379 Data size: 88080 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -175,7 +175,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: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 379 Data size: 88080 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col3 (type: boolean), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: float), _col10 (type: float), _col11 (type: float), _col12 (type: double), _col13 (type: double), _col14 (type: bigint), _col15 (type: double), _col16 (type: double), _col17 (type: double), _col18 (type: double), _col19 (type: double), _col20 (type: double), _col21 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -195,10 +195,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey3 (type: timestamp), KEY.reducesinkkey1 (type: float), KEY.reducesinkkey0 (type: string), VALUE._col0 (type: boolean), KEY.reducesinkkey2 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double), VALUE._col3 (type: double), VALUE._col4 (type: float), VALUE._col5 (type: float), VALUE._col6 (type: float), VALUE._col7 (type: float), VALUE._col8 (type: double), VALUE._col9 (type: double), VALUE._col10 (type: bigint), VALUE._col11 (type: d [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21
-          Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 379 Data size: 88080 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 379 Data size: 88080 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/parquet_vectorization_15.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_15.q.out
index f2d32b3..92d5961 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_15.q.out
@@ -80,7 +80,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: ((cstring1 like '10%') or (cstring2 like '%ss%') or ((cint >= -75) and (UDFToShort(ctinyint) = csmallint) and (cdouble >= -3728.0D))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -89,7 +89,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterStringColLikeStringScalar(col 6:string, pattern 10%), FilterStringColLikeStringScalar(col 7:string, pattern %ss%), FilterExprAndExpr(children: FilterLongColGreaterEqualLongScalar(col 2:int, val -75), FilterLongColEqualLongColumn(col 0:smallint, col 1:smallint)(children: col 0:tinyint), FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -3728.0)))
               predicate: ((cstring1 like '10%') or (cstring2 like '%ss%') or ((cint >= -75) and (UDFToShort(ctinyint) = csmallint) and (cdouble >= -3728.0D))) (type: boolean)
-              Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cfloat (type: float), cboolean1 (type: boolean), cdouble (type: double), cstring1 (type: string), ctinyint (type: tinyint), cint (type: int), ctimestamp1 (type: timestamp), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble(cint)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -98,7 +98,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [4, 10, 5, 6, 0, 2, 8, 4, 13, 14, 17, 18, 21]
                     selectExpressions: DoubleColMultiplyDoubleColumn(col 4:double, col 4:double)(children: col 4:float, col 4:float) -> 13:double, CastLongToDouble(col 0:tinyint) -> 14:double, DoubleColMultiplyDoubleColumn(col 15:double, col 16:double)(children: CastLongToDouble(col 0:tinyint) -> 15:double, CastLongToDouble(col 0:tinyint) -> 16:double) -> 17:double, CastLongToDouble(col 2:int) -> 18:double, DoubleColMultiplyDoubleColumn(col 19:double, col 20:double)(children: CastLongToD [...]
-                Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col8), sum(_col7), count(_col0), min(_col2), sum(_col10), sum(_col9), count(_col4), sum(_col12), sum(_col11), count(_col5)
                   Group By Vectorization:
@@ -113,7 +113,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
-                  Statistics: Num rows: 6144 Data size: 1278652 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6144 Data size: 1216372 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
                     sort order: +++++++
@@ -123,7 +123,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: 6144 Data size: 1278652 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6144 Data size: 1216372 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col7 (type: double), _col8 (type: double), _col9 (type: bigint), _col10 (type: double), _col11 (type: double), _col12 (type: double), _col13 (type: bigint), _col14 (type: double), _col15 (type: double), _col16 (type: bigint)
       Execution mode: vectorized
       Map Vectorization:
@@ -144,11 +144,11 @@ STAGE PLANS:
           keys: KEY._col0 (type: float), KEY._col1 (type: boolean), KEY._col2 (type: double), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int), KEY._col6 (type: timestamp)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
-          Statistics: Num rows: 3072 Data size: 639332 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6144 Data size: 1216372 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp), power(((_col7 - ((_col8 * _col8) / _col9)) / CASE WHEN ((_col9 = 1L)) THEN (null) ELSE ((_col9 - 1)) END), 0.5) (type: double), (-26.28 - CAST( _col5 AS decimal(10,0))) (type: decimal(13,2)), _col10 (type: double), (_col2 * 79.553D) (type: double), (33.0 % _col0) (type: float), power(((_col11 - ((_col12 * _col1 [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-            Statistics: Num rows: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6144 Data size: 2592628 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -170,7 +170,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: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 2592628 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col7 (type: double), _col8 (type: decimal(13,2)), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: double), _col13 (type: double), _col14 (type: double), _col15 (type: tinyint), _col16 (type: double), _col17 (type: float), _col18 (type: int), _col19 (type: decimal(13,2)), _col20 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -189,10 +189,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: float), KEY.reducesinkkey1 (type: boolean), KEY.reducesinkkey2 (type: double), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: tinyint), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey6 (type: timestamp), VALUE._col0 (type: double), VALUE._col1 (type: decimal(13,2)), VALUE._col2 (type: double), VALUE._col3 (type: double), VALUE._col4 (type: float), VALUE._col5 (type: double), VALUE._col6 (type: double), VALUE._col7 (type: do [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-          Statistics: Num rows: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6144 Data size: 2592628 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6144 Data size: 2592628 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/parquet_vectorization_16.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_16.q.out
index ae684e6..a3981eb 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_16.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_16.q.out
@@ -57,7 +57,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2308074 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2183514 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -66,7 +66,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterStringColLikeStringScalar(col 7:string, pattern %b%), FilterExprOrExpr(children: FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -1.389), FilterStringGroupColLessStringScalar(col 6:string, val a)))
               predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp), (cdouble * cdouble) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3
@@ -75,7 +75,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [6, 5, 8, 13]
                     selectExpressions: DoubleColMultiplyDoubleColumn(col 5:double, col 5:double) -> 13:double
-                Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(_col1), sum(_col3), sum(_col1), min(_col1)
                   Group By Vectorization:
@@ -90,7 +90,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                  Statistics: Num rows: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
                     sort order: +++
@@ -100,7 +100,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: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col3 (type: bigint), _col4 (type: double), _col5 (type: double), _col6 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -122,14 +122,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-          Statistics: Num rows: 1536 Data size: 227586 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639D) (type: double), (- (_col1 - 9763215.5639D)) (type: double), _col3 (type: bigint), power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5) (type: double), (- power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5)) (type: double), (power(((_col4 - ((_col5 * _col5 [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-            Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 3072 Data size: 890996 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3072 Data size: 890996 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/parquet_vectorization_17.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_17.q.out
index 6adb128..5fed7fc 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_17.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_17.q.out
@@ -65,7 +65,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: ((cbigint > -23L) and ((ctinyint >= 33Y) or (UDFToLong(csmallint) >= cbigint) or (UDFToDouble(cfloat) = cdouble)) and ((cdouble <> 988888.0D) or (CAST( cint AS decimal(13,3)) > -863.257))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1647550 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1522990 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -74,7 +74,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterLongColGreaterLongScalar(col 3:bigint, val -23), FilterExprOrExpr(children: FilterLongColGreaterEqualLongScalar(col 0:tinyint, val 33), FilterLongColGreaterEqualLongColumn(col 1:bigint, col 3:bigint)(children: col 1:smallint), FilterDoubleColEqualDoubleColumn(col 4:double, col 5:double)(children: col 4:float)), FilterExprOrExpr(children: FilterDoubleColNotEqualDoubleScalar(col 5:double, val 988888.0), FilterDecimalC [...]
               predicate: ((cbigint > -23L) and ((ctinyint >= 33Y) or (UDFToLong(csmallint) >= cbigint) or (UDFToDouble(cfloat) = cdouble)) and ((cdouble <> 988888.0D) or (CAST( cint AS decimal(13,3)) > -863.257))) (type: boolean)
-              Statistics: Num rows: 6141 Data size: 823456 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6141 Data size: 761216 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cfloat (type: float), cstring1 (type: string), cint (type: int), ctimestamp1 (type: timestamp), cdouble (type: double), cbigint (type: bigint), (UDFToDouble(cfloat) / UDFToDouble(ctinyint)) (type: double), (UDFToLong(cint) % cbigint) (type: bigint), (- cdouble) (type: double), (cdouble + (UDFToDouble(cfloat) / UDFToDouble(ctinyint))) (type: double), (cdouble / UDFToDouble(cint)) (type: double), (- (- cdouble)) (type: double), (9763215.5639 % CAST( cbigint AS  [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -83,7 +83,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [4, 6, 2, 8, 5, 3, 15, 16, 17, 20, 22, 24, 26, 29]
                     selectExpressions: DoubleColDivideDoubleColumn(col 4:double, col 14:double)(children: col 4:float, CastLongToDouble(col 0:tinyint) -> 14:double) -> 15:double, LongColModuloLongColumn(col 2:bigint, col 3:bigint)(children: col 2:int) -> 16:bigint, DoubleColUnaryMinus(col 5:double) -> 17:double, DoubleColAddDoubleColumn(col 5:double, col 19:double)(children: DoubleColDivideDoubleColumn(col 4:double, col 18:double)(children: col 4:float, CastLongToDouble(col 0:tinyint) -> [...]
-                Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6141 Data size: 1756220 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col5 (type: bigint), _col0 (type: float)
                   sort order: ++
@@ -92,7 +92,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: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6141 Data size: 1756220 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: timestamp), _col4 (type: double), _col6 (type: double), _col7 (type: bigint), _col8 (type: double), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: decimal(11,4)), _col13 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -112,10 +112,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey1 (type: float), VALUE._col0 (type: string), VALUE._col1 (type: int), VALUE._col2 (type: timestamp), VALUE._col3 (type: double), KEY.reducesinkkey0 (type: bigint), VALUE._col4 (type: double), VALUE._col5 (type: bigint), VALUE._col6 (type: double), VALUE._col7 (type: double), VALUE._col8 (type: double), VALUE._col9 (type: double), VALUE._col10 (type: decimal(11,4)), VALUE._col11 (type: double)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
-          Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6141 Data size: 1756220 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6141 Data size: 1756220 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/parquet_vectorization_2.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_2.q.out
index 1b2800f..8ac5aaf 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_2.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_2.q.out
@@ -63,7 +63,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (((cdouble < UDFToDouble(ctinyint)) and ((UDFToDouble(ctimestamp2) <> -10669.0D) or (cint < 359))) or ((ctimestamp1 < ctimestamp2) and (cstring2 like 'b%') and (cfloat <= -5638.15))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2157324 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1908244 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -72,7 +72,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleColumn(col 5:double, col 13:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterExprOrExpr(children: FilterDoubleColNotEqualDoubleScalar(col 14:double, val -10669.0)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), FilterLongColLessLongScalar(col 2:int, val 359))), FilterExprAndExpr(children: FilterTimestampColLessTimestampColumn(col 8:t [...]
               predicate: (((cdouble < UDFToDouble(ctinyint)) and ((UDFToDouble(ctimestamp2) <> -10669.0D) or (cint < 359))) or ((ctimestamp1 < ctimestamp2) and (cstring2 like 'b%') and (cfloat <= -5638.15))) (type: boolean)
-              Statistics: Num rows: 4096 Data size: 719232 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 4096 Data size: 636272 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: csmallint (type: smallint), cfloat (type: float), cbigint (type: bigint), ctinyint (type: tinyint), cdouble (type: double), UDFToDouble(cbigint) (type: double), (UDFToDouble(cbigint) * UDFToDouble(cbigint)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
@@ -81,7 +81,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [1, 4, 3, 0, 5, 15, 18]
                     selectExpressions: CastLongToDouble(col 3:bigint) -> 15:double, DoubleColMultiplyDoubleColumn(col 16:double, col 17:double)(children: CastLongToDouble(col 3:bigint) -> 16:double, CastLongToDouble(col 3:bigint) -> 17:double) -> 18:double
-                Statistics: Num rows: 4096 Data size: 719232 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4096 Data size: 636272 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col0), count(_col0), sum(_col1), sum(_col6), sum(_col5), count(_col2), count(), min(_col3), sum(_col4), count(_col4)
                   Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/parquet_vectorization_3.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_3.q.out
index 4d3f0b6..1c87b71 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_3.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_3.q.out
@@ -68,7 +68,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (((UDFToFloat(cint) <= cfloat) and (CAST( cbigint AS decimal(22,3)) <> 79.553) and (UDFToDouble(ctimestamp2) = -29071.0D)) or ((UDFToDouble(cbigint) > cdouble) and (CAST( csmallint AS decimal(8,3)) >= 79.553) and (ctimestamp1 > ctimestamp2))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1276620 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1027540 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -77,7 +77,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleColumn(col 13:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 13:float), FilterDecimalColNotEqualDecimalScalar(col 14:decimal(22,3), val 79.553)(children: CastLongToDecimal(col 3:bigint) -> 14:decimal(22,3)), FilterDoubleColEqualDoubleScalar(col 15:double, val -29071.0)(children: CastTimestampToDouble(col 9:timestamp) -> 15:double)), Filte [...]
               predicate: (((UDFToFloat(cint) <= cfloat) and (CAST( cbigint AS decimal(22,3)) <> 79.553) and (UDFToDouble(ctimestamp2) = -29071.0D)) or ((UDFToDouble(cbigint) > cdouble) and (CAST( csmallint AS decimal(8,3)) >= 79.553) and (ctimestamp1 > ctimestamp2))) (type: boolean)
-              Statistics: Num rows: 2503 Data size: 260060 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2503 Data size: 209380 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: csmallint (type: smallint), ctinyint (type: tinyint), cfloat (type: float), cint (type: int), UDFToDouble(csmallint) (type: double), (UDFToDouble(csmallint) * UDFToDouble(csmallint)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
@@ -86,7 +86,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [1, 0, 4, 2, 18, 21, 22, 25, 4, 26, 27, 30]
                     selectExpressions: CastLongToDouble(col 1:smallint) -> 18:double, DoubleColMultiplyDoubleColumn(col 19:double, col 20:double)(children: CastLongToDouble(col 1:smallint) -> 19:double, CastLongToDouble(col 1:smallint) -> 20:double) -> 21:double, CastLongToDouble(col 0:tinyint) -> 22:double, DoubleColMultiplyDoubleColumn(col 23:double, col 24:double)(children: CastLongToDouble(col 0:tinyint) -> 23:double, CastLongToDouble(col 0:tinyint) -> 24:double) -> 25:double, Double [...]
-                Statistics: Num rows: 2503 Data size: 260060 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2503 Data size: 209380 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col5), sum(_col4), count(_col0), sum(_col7), sum(_col6), count(_col1), sum(_col9), sum(_col8), count(_col2), sum(_col2), sum(_col3), count(_col3), sum(_col11), sum(_col10)
                   Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/parquet_vectorization_5.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_5.q.out
index f100fff..9e99c47 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_5.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_5.q.out
@@ -57,7 +57,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (((cstring1 like '%b%') and cboolean2 is not null) or ((UDFToDouble(ctinyint) = cdouble) and (cstring2 like 'a') and ctimestamp2 is not null)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2454862 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2330342 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -66,7 +66,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterStringColLikeStringScalar(col 6:string, pattern %b%), SelectColumnIsNotNull(col 11:boolean)), FilterExprAndExpr(children: FilterDoubleColEqualDoubleColumn(col 13:double, col 5:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterStringColLikeStringScalar(col 7:string, pattern a), SelectColumnIsNotNull(col 9:timestamp)))
               predicate: (((cstring1 like '%b%') and cboolean2 is not null) or ((UDFToDouble(ctinyint) = cdouble) and (cstring2 like 'a') and ctimestamp2 is not null)) (type: boolean)
-              Statistics: Num rows: 7658 Data size: 1529972 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6879 Data size: 1304690 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int)
                 outputColumnNames: ctinyint, csmallint, cint
@@ -74,7 +74,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2]
-                Statistics: Num rows: 7658 Data size: 1529972 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6879 Data size: 1304690 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: max(csmallint), count(), min(csmallint), sum(cint), max(ctinyint)
                   Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/parquet_vectorization_7.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_7.q.out
index d1a1ae4..3a8c788 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_7.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_7.q.out
@@ -71,7 +71,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28815.0D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 3019778 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2770698 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -80,7 +80,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterDoubleColGreaterDoubleScalar(col 5:double, val 988888.0), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28815.0)(children: CastTimestampToDouble(col 9:timestamp) -> 13:double), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 3569.0))), FilterExprOrExpr(children: FilterDoubleColLessEqualDoubleScalar(col 14:double, val -28800.0)(children: CastTimes [...]
               predicate: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28815.0D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-              Statistics: Num rows: 11033 Data size: 2711364 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 11033 Data size: 2487724 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cboolean1 (type: boolean), cbigint (type: bigint), csmallint (type: smallint), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cstring1 (type: string), (cbigint + cbigint) (type: bigint), (UDFToInteger(csmallint) % -257) (type: int), (- csmallint) (type: smallint), (- ctinyint) (type: tinyint), (UDFToInteger((- ctinyint)) + 17) (type: int), (cbigint * UDFToLong((- csmallint))) (type: bigint), (cint % UDFToInteger(csmallint)) (type: int), (- ctinyint) [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -89,7 +89,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [10, 3, 1, 0, 8, 6, 15, 16, 17, 18, 20, 22, 23, 24, 26]
                     selectExpressions: LongColAddLongColumn(col 3:bigint, col 3:bigint) -> 15:bigint, LongColModuloLongScalar(col 1:int, val -257)(children: col 1:smallint) -> 16:int, LongColUnaryMinus(col 1:smallint) -> 17:smallint, LongColUnaryMinus(col 0:tinyint) -> 18:tinyint, LongColAddLongScalar(col 19:int, val 17)(children: LongColUnaryMinus(col 0:tinyint) -> 19:tinyint) -> 20:int, LongColMultiplyLongColumn(col 3:bigint, col 21:bigint)(children: LongColUnaryMinus(col 1:smallint) - [...]
-                Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: boolean), _col1 (type: bigint), _col2 (type: smallint), _col3 (type: tinyint), _col4 (type: timestamp), _col5 (type: string), _col6 (type: bigint), _col7 (type: int), _col8 (type: smallint), _col9 (type: tinyint), _col10 (type: int), _col11 (type: bigint), _col12 (type: int), _col13 (type: tinyint), _col14 (type: tinyint)
                   sort order: +++++++++++++++
@@ -98,7 +98,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: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -118,13 +118,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: boolean), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: smallint), KEY.reducesinkkey3 (type: tinyint), KEY.reducesinkkey4 (type: timestamp), KEY.reducesinkkey5 (type: string), KEY.reducesinkkey6 (type: bigint), KEY.reducesinkkey7 (type: int), KEY.reducesinkkey8 (type: smallint), KEY.reducesinkkey9 (type: tinyint), KEY.reducesinkkey10 (type: int), KEY.reducesinkkey11 (type: bigint), KEY.reducesinkkey12 (type: int), KEY.reduce [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-          Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 25
-            Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 25 Data size: 4180 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 25 Data size: 4180 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -290,7 +290,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28792.315D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 3019778 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2770698 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -299,7 +299,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterDoubleColGreaterDoubleScalar(col 5:double, val 988888.0), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28792.315)(children: CastTimestampToDouble(col 9:timestamp) -> 13:double), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 3569.0))), FilterExprOrExpr(children: FilterDoubleColLessEqualDoubleScalar(col 14:double, val -28800.0)(children: CastTim [...]
               predicate: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28792.315D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-              Statistics: Num rows: 11033 Data size: 2711364 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 11033 Data size: 2487724 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cboolean1 (type: boolean), cbigint (type: bigint), csmallint (type: smallint), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cstring1 (type: string), (cbigint + cbigint) (type: bigint), (UDFToInteger(csmallint) % -257) (type: int), (- csmallint) (type: smallint), (- ctinyint) (type: tinyint), (UDFToInteger((- ctinyint)) + 17) (type: int), (cbigint * UDFToLong((- csmallint))) (type: bigint), (cint % UDFToInteger(csmallint)) (type: int), (- ctinyint) [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -308,7 +308,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [10, 3, 1, 0, 8, 6, 15, 16, 17, 18, 20, 22, 23, 24, 26]
                     selectExpressions: LongColAddLongColumn(col 3:bigint, col 3:bigint) -> 15:bigint, LongColModuloLongScalar(col 1:int, val -257)(children: col 1:smallint) -> 16:int, LongColUnaryMinus(col 1:smallint) -> 17:smallint, LongColUnaryMinus(col 0:tinyint) -> 18:tinyint, LongColAddLongScalar(col 19:int, val 17)(children: LongColUnaryMinus(col 0:tinyint) -> 19:tinyint) -> 20:int, LongColMultiplyLongColumn(col 3:bigint, col 21:bigint)(children: LongColUnaryMinus(col 1:smallint) - [...]
-                Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: boolean), _col1 (type: bigint), _col2 (type: smallint), _col3 (type: tinyint), _col4 (type: timestamp), _col5 (type: string), _col6 (type: bigint), _col7 (type: int), _col8 (type: smallint), _col9 (type: tinyint), _col10 (type: int), _col11 (type: bigint), _col12 (type: int), _col13 (type: tinyint), _col14 (type: tinyint)
                   sort order: +++++++++++++++
@@ -317,7 +317,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: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -337,13 +337,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: boolean), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: smallint), KEY.reducesinkkey3 (type: tinyint), KEY.reducesinkkey4 (type: timestamp), KEY.reducesinkkey5 (type: string), KEY.reducesinkkey6 (type: bigint), KEY.reducesinkkey7 (type: int), KEY.reducesinkkey8 (type: smallint), KEY.reducesinkkey9 (type: tinyint), KEY.reducesinkkey10 (type: int), KEY.reducesinkkey11 (type: bigint), KEY.reducesinkkey12 (type: int), KEY.reduce [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-          Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 25
-            Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 25 Data size: 4180 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 25 Data size: 4180 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/parquet_vectorization_8.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_8.q.out
index b15b145..afc9c8e 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_8.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_8.q.out
@@ -67,7 +67,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 10.0D) and (UDFToDouble(ctimestamp2) <> 16.0D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2983078 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2733998 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -76,7 +76,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val -6432.0), FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleScalar(col 13:double, val 10.0)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val 16.0)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), SelectColumnIsNotNull(col 7:string)), FilterExprAndExpr(children: FilterDoubleColEqualDo [...]
               predicate: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 10.0D) and (UDFToDouble(ctimestamp2) <> 16.0D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-              Statistics: Num rows: 3059 Data size: 742850 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3059 Data size: 680930 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ctimestamp1 (type: timestamp), cdouble (type: double), cboolean1 (type: boolean), cstring1 (type: string), cfloat (type: float), (- cdouble) (type: double), (-5638.15D - cdouble) (type: double), (cdouble * -257.0D) (type: double), (UDFToFloat(cint) + cfloat) (type: float), ((- cdouble) + UDFToDouble(cbigint)) (type: double), (- cdouble) (type: double), (-1.389 - cfloat) (type: float), (- cfloat) (type: float), ((-5638.15D - cdouble) + UDFToDouble((UDFToFloat( [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -85,7 +85,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [8, 5, 10, 6, 4, 15, 16, 17, 19, 22, 23, 24, 25, 29]
                     selectExpressions: DoubleColUnaryMinus(col 5:double) -> 15:double, DoubleScalarSubtractDoubleColumn(val -5638.15, col 5:double) -> 16:double, DoubleColMultiplyDoubleScalar(col 5:double, val -257.0) -> 17:double, DoubleColAddDoubleColumn(col 18:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 18:float) -> 19:float, DoubleColAddDoubleColumn(col 20:double, col 21:double)(children: DoubleColUnaryMinus(col 5:double) -> 20:double, CastLongToDouble( [...]
-                Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: timestamp), _col1 (type: double), _col2 (type: boolean), _col3 (type: string), _col4 (type: float), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: float), _col13 (type: double)
                   sort order: ++++++++++++++
@@ -94,7 +94,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: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -114,13 +114,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: timestamp), KEY.reducesinkkey1 (type: double), KEY.reducesinkkey2 (type: boolean), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: float), KEY.reducesinkkey5 (type: double), KEY.reducesinkkey6 (type: double), KEY.reducesinkkey7 (type: double), KEY.reducesinkkey8 (type: float), KEY.reducesinkkey9 (type: double), KEY.reducesinkkey5 (type: double), KEY.reducesinkkey11 (type: float), KEY.reducesinkkey12 (type: float), KEY.reducesi [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
-          Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 20
-            Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3600 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 3600 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -273,7 +273,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 12.503D) and (UDFToDouble(ctimestamp2) <> 11.998D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2983078 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2733998 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -282,7 +282,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val -6432.0), FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleScalar(col 13:double, val 12.503)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val 11.998)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), SelectColumnIsNotNull(col 7:string)), FilterExprAndExpr(children: FilterDoubleColEqu [...]
               predicate: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 12.503D) and (UDFToDouble(ctimestamp2) <> 11.998D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-              Statistics: Num rows: 3059 Data size: 742850 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3059 Data size: 680930 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ctimestamp1 (type: timestamp), cdouble (type: double), cboolean1 (type: boolean), cstring1 (type: string), cfloat (type: float), (- cdouble) (type: double), (-5638.15D - cdouble) (type: double), (cdouble * -257.0D) (type: double), (UDFToFloat(cint) + cfloat) (type: float), ((- cdouble) + UDFToDouble(cbigint)) (type: double), (- cdouble) (type: double), (-1.389 - cfloat) (type: float), (- cfloat) (type: float), ((-5638.15D - cdouble) + UDFToDouble((UDFToFloat( [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -291,7 +291,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [8, 5, 10, 6, 4, 15, 16, 17, 19, 22, 23, 24, 25, 29]
                     selectExpressions: DoubleColUnaryMinus(col 5:double) -> 15:double, DoubleScalarSubtractDoubleColumn(val -5638.15, col 5:double) -> 16:double, DoubleColMultiplyDoubleScalar(col 5:double, val -257.0) -> 17:double, DoubleColAddDoubleColumn(col 18:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 18:float) -> 19:float, DoubleColAddDoubleColumn(col 20:double, col 21:double)(children: DoubleColUnaryMinus(col 5:double) -> 20:double, CastLongToDouble( [...]
-                Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: timestamp), _col1 (type: double), _col2 (type: boolean), _col3 (type: string), _col4 (type: float), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: float), _col13 (type: double)
                   sort order: ++++++++++++++
@@ -300,7 +300,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: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -320,13 +320,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: timestamp), KEY.reducesinkkey1 (type: double), KEY.reducesinkkey2 (type: boolean), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: float), KEY.reducesinkkey5 (type: double), KEY.reducesinkkey6 (type: double), KEY.reducesinkkey7 (type: double), KEY.reducesinkkey8 (type: float), KEY.reducesinkkey9 (type: double), KEY.reducesinkkey5 (type: double), KEY.reducesinkkey11 (type: float), KEY.reducesinkkey12 (type: float), KEY.reducesi [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
-          Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 20
-            Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3600 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 3600 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/parquet_vectorization_9.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_9.q.out
index ae684e6..a3981eb 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_9.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_9.q.out
@@ -57,7 +57,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesparquet
             filterExpr: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2308074 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2183514 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -66,7 +66,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterStringColLikeStringScalar(col 7:string, pattern %b%), FilterExprOrExpr(children: FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -1.389), FilterStringGroupColLessStringScalar(col 6:string, val a)))
               predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp), (cdouble * cdouble) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3
@@ -75,7 +75,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [6, 5, 8, 13]
                     selectExpressions: DoubleColMultiplyDoubleColumn(col 5:double, col 5:double) -> 13:double
-                Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(_col1), sum(_col3), sum(_col1), min(_col1)
                   Group By Vectorization:
@@ -90,7 +90,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                  Statistics: Num rows: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
                     sort order: +++
@@ -100,7 +100,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: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col3 (type: bigint), _col4 (type: double), _col5 (type: double), _col6 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -122,14 +122,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-          Statistics: Num rows: 1536 Data size: 227586 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639D) (type: double), (- (_col1 - 9763215.5639D)) (type: double), _col3 (type: bigint), power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5) (type: double), (- power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5)) (type: double), (power(((_col4 - ((_col5 * _col5 [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-            Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 3072 Data size: 890996 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3072 Data size: 890996 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/partitioned_table_stats.q.out b/ql/src/test/results/clientpositive/partitioned_table_stats.q.out
index 4376bed..0ce2d3e 100644
--- a/ql/src/test/results/clientpositive/partitioned_table_stats.q.out
+++ b/ql/src/test/results/clientpositive/partitioned_table_stats.q.out
@@ -198,8 +198,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats_n0
 col_name            	ts                  
 data_type           	timestamp           
-min                 	1325379723          
-max                 	1325379723          
+min                 	2012-01-01 01:02:03 
+max                 	2012-01-01 01:02:03 
 num_nulls           	1                   
 distinct_count      	1                   
 avg_col_len         	                    
@@ -485,8 +485,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats_n0
 col_name            	ts                  
 data_type           	timestamp           
-min                 	1325379722          
-max                 	1325379723          
+min                 	2012-01-01 01:02:02 
+max                 	2012-01-01 01:02:03 
 num_nulls           	1                   
 distinct_count      	2                   
 avg_col_len         	                    
@@ -772,8 +772,8 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@datatype_stats_n0
 col_name            	ts                  
 data_type           	timestamp           
-min                 	1325379722          
-max                 	1325379724          
+min                 	2012-01-01 01:02:02 
+max                 	2012-01-01 01:02:04 
 num_nulls           	1                   
 distinct_count      	3                   
 avg_col_len         	                    
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
index 2ef7b13..6ade6af 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
@@ -767,7 +767,7 @@ Stage-3
                     Stage-1
                       Reducer 2
                       File Output Operator [FS_9]
-                        Group By Operator [GBY_7] (rows=1/1 width=2760)
+                        Group By Operator [GBY_7] (rows=1/1 width=2824)
                           Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["compute_stats(VALUE._col0, 'hll')","compute_stats(VALUE._col2, 'hll')","compute_stats(VALUE._col3, 'hll')","compute_stats(VALUE._col4, 'hll')","compute_stats(VALUE._col5, 'hll')"]
                         <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized
                           File Output Operator [FS_14]
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
index 16ecfaa..4eb698e 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
@@ -48,29 +48,29 @@ Stage-0
     Stage-1
       Reducer 3
       File Output Operator [FS_12]
-        Select Operator [SEL_11] (rows=4626/10 width=552)
+        Select Operator [SEL_11] (rows=4626/10 width=528)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"]
         <-Reducer 2 [SIMPLE_EDGE]
           SHUFFLE [RS_10]
-            Merge Join Operator [MERGEJOIN_27] (rows=4626/10 width=552)
+            Merge Join Operator [MERGEJOIN_27] (rows=4626/10 width=528)
               Conds:RS_6._col2=RS_7._col2(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"]
             <-Map 1 [SIMPLE_EDGE]
               SHUFFLE [RS_6]
                 PartitionCols:_col2
-                Select Operator [SEL_2] (rows=3078/10 width=251)
+                Select Operator [SEL_2] (rows=3078/10 width=231)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
-                  Filter Operator [FIL_15] (rows=3078/10 width=251)
+                  Filter Operator [FIL_15] (rows=3078/10 width=231)
                     predicate:cint BETWEEN 1000000 AND 3000000
-                    TableScan [TS_0] (rows=12288/12288 width=251)
+                    TableScan [TS_0] (rows=12288/12288 width=231)
                       default@alltypesorc,a,Tbl:COMPLETE,Col:COMPLETE,Output:["ctinyint","csmallint","cint","cbigint","cfloat","cdouble","cstring1","cstring2","ctimestamp1","ctimestamp2","cboolean1","cboolean2"]
             <-Map 4 [SIMPLE_EDGE]
               SHUFFLE [RS_7]
                 PartitionCols:_col2
-                Select Operator [SEL_5] (rows=2298/10 width=251)
+                Select Operator [SEL_5] (rows=2298/10 width=231)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
-                  Filter Operator [FIL_16] (rows=2298/10 width=251)
+                  Filter Operator [FIL_16] (rows=2298/10 width=231)
                     predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
-                    TableScan [TS_3] (rows=12288/12288 width=251)
+                    TableScan [TS_3] (rows=12288/12288 width=231)
                       default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["ctinyint","csmallint","cint","cbigint","cfloat","cdouble","cstring1","cstring2","ctimestamp1","ctimestamp2","cboolean1","cboolean2"]
 
 PREHOOK: query: select
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
index fa85521..da57efe 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
@@ -615,7 +615,7 @@ Stage-3
                     Stage-1
                       Reducer 2
                       File Output Operator [FS_10]
-                        Group By Operator [GBY_8] (rows=1 width=2760)
+                        Group By Operator [GBY_8] (rows=1 width=2824)
                           Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["compute_stats(VALUE._col0)","compute_stats(VALUE._col1)","compute_stats(VALUE._col2)","compute_stats(VALUE._col3)","compute_stats(VALUE._col4)"]
                         <-Map 1 [CUSTOM_SIMPLE_EDGE]
                           File Output Operator [FS_3]
@@ -627,7 +627,7 @@ Stage-3
                                 TableScan [TS_0] (rows=1 width=352)
                                   default@orc_merge5_n0,orc_merge5_n0,Tbl:COMPLETE,Col:NONE,Output:["userid","string1","subtype","decimal1","ts"]
                           PARTITION_ONLY_SHUFFLE [RS_7]
-                            Group By Operator [GBY_6] (rows=1 width=2696)
+                            Group By Operator [GBY_6] (rows=1 width=2760)
                               Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["compute_stats(userid, 'hll')","compute_stats(string1, 'hll')","compute_stats(subtype, 'hll')","compute_stats(decimal1, 'hll')","compute_stats(ts, 'hll')"]
                               Select Operator [SEL_5] (rows=1 width=352)
                                 Output:["userid","string1","subtype","decimal1","ts"]
diff --git a/ql/src/test/results/clientpositive/timestamp_comparison3.q.out b/ql/src/test/results/clientpositive/timestamp_comparison3.q.out
new file mode 100644
index 0000000..3977be7
--- /dev/null
+++ b/ql/src/test/results/clientpositive/timestamp_comparison3.q.out
@@ -0,0 +1,212 @@
+PREHOOK: query: create database timestamp_test_n123
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:timestamp_test_n123
+POSTHOOK: query: create database timestamp_test_n123
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:timestamp_test_n123
+PREHOOK: query: create table timestamp_test_n123.onecolumntable (ts timestamp)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:timestamp_test_n123
+PREHOOK: Output: timestamp_test_n123@onecolumntable
+POSTHOOK: query: create table timestamp_test_n123.onecolumntable (ts timestamp)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:timestamp_test_n123
+POSTHOOK: Output: timestamp_test_n123@onecolumntable
+PREHOOK: query: insert into timestamp_test_n123.onecolumntable values
+('2015-01-01 00:00:00'),
+('2015-01-02 00:00:00'),
+('2015-01-03 00:00:00'),
+('2015-01-04 00:00:00'),
+('2015-01-05 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: timestamp_test_n123@onecolumntable
+POSTHOOK: query: insert into timestamp_test_n123.onecolumntable values
+('2015-01-01 00:00:00'),
+('2015-01-02 00:00:00'),
+('2015-01-03 00:00:00'),
+('2015-01-04 00:00:00'),
+('2015-01-05 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: timestamp_test_n123@onecolumntable
+POSTHOOK: Lineage: onecolumntable.ts SCRIPT []
+PREHOOK: query: describe formatted timestamp_test_n123.onecolumntable ts
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: timestamp_test_n123@onecolumntable
+POSTHOOK: query: describe formatted timestamp_test_n123.onecolumntable ts
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: timestamp_test_n123@onecolumntable
+col_name            	ts                  
+data_type           	timestamp           
+min                 	2015-01-01 00:00:00 
+max                 	2015-01-05 00:00:00 
+num_nulls           	0                   
+distinct_count      	5                   
+avg_col_len         	                    
+max_col_len         	                    
+num_trues           	                    
+num_falses          	                    
+bit_vector          	HL                  
+comment             	from deserializer   
+COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"ts\":\"true\"}}
+PREHOOK: query: explain
+select ts from timestamp_test_n123.onecolumntable
+where ts >= cast('2015-01-02 00:00:00' as timestamp)
+  and ts <= cast('2015-01-04 00:00:00' as timestamp)
+PREHOOK: type: QUERY
+PREHOOK: Input: timestamp_test_n123@onecolumntable
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select ts from timestamp_test_n123.onecolumntable
+where ts >= cast('2015-01-02 00:00:00' as timestamp)
+  and ts <= cast('2015-01-04 00:00:00' as timestamp)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: timestamp_test_n123@onecolumntable
+#### 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
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: onecolumntable
+            filterExpr: ts BETWEEN TIMESTAMP'2015-01-02 00:00:00' AND TIMESTAMP'2015-01-04 00:00:00' (type: boolean)
+            Statistics: Num rows: 5 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: ts BETWEEN TIMESTAMP'2015-01-02 00:00:00' AND TIMESTAMP'2015-01-04 00:00:00' (type: boolean)
+              Statistics: Num rows: 3 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: ts (type: timestamp)
+                outputColumnNames: _col0
+                Statistics: Num rows: 3 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 3 Data size: 120 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
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain
+select ts from timestamp_test_n123.onecolumntable
+where ts >= cast('2015-01-02 00:00:00' as timestamp)
+  and ts <= cast('2015-01-03 00:00:00' as timestamp)
+PREHOOK: type: QUERY
+PREHOOK: Input: timestamp_test_n123@onecolumntable
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select ts from timestamp_test_n123.onecolumntable
+where ts >= cast('2015-01-02 00:00:00' as timestamp)
+  and ts <= cast('2015-01-03 00:00:00' as timestamp)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: timestamp_test_n123@onecolumntable
+#### 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
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: onecolumntable
+            filterExpr: ts BETWEEN TIMESTAMP'2015-01-02 00:00:00' AND TIMESTAMP'2015-01-03 00:00:00' (type: boolean)
+            Statistics: Num rows: 5 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: ts BETWEEN TIMESTAMP'2015-01-02 00:00:00' AND TIMESTAMP'2015-01-03 00:00:00' (type: boolean)
+              Statistics: Num rows: 2 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: ts (type: timestamp)
+                outputColumnNames: _col0
+                Statistics: Num rows: 2 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 2 Data size: 80 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
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain
+select ts from timestamp_test_n123.onecolumntable
+where ts >= cast('2015-01-01 00:00:00' as timestamp)
+  and ts <= cast('2015-01-08 00:00:00' as timestamp)
+PREHOOK: type: QUERY
+PREHOOK: Input: timestamp_test_n123@onecolumntable
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select ts from timestamp_test_n123.onecolumntable
+where ts >= cast('2015-01-01 00:00:00' as timestamp)
+  and ts <= cast('2015-01-08 00:00:00' as timestamp)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: timestamp_test_n123@onecolumntable
+#### 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
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: onecolumntable
+            filterExpr: ts BETWEEN TIMESTAMP'2015-01-01 00:00:00' AND TIMESTAMP'2015-01-08 00:00:00' (type: boolean)
+            Statistics: Num rows: 5 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: ts BETWEEN TIMESTAMP'2015-01-01 00:00:00' AND TIMESTAMP'2015-01-08 00:00:00' (type: boolean)
+              Statistics: Num rows: 5 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: ts (type: timestamp)
+                outputColumnNames: _col0
+                Statistics: Num rows: 5 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 5 Data size: 200 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
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: drop table timestamp_test_n123.onecolumntable
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: timestamp_test_n123@onecolumntable
+PREHOOK: Output: timestamp_test_n123@onecolumntable
+POSTHOOK: query: drop table timestamp_test_n123.onecolumntable
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: timestamp_test_n123@onecolumntable
+POSTHOOK: Output: timestamp_test_n123@onecolumntable
+PREHOOK: query: drop database timestamp_test_n123
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:timestamp_test_n123
+PREHOOK: Output: database:timestamp_test_n123
+POSTHOOK: query: drop database timestamp_test_n123
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:timestamp_test_n123
+POSTHOOK: Output: database:timestamp_test_n123
diff --git a/ql/src/test/results/clientpositive/timestamp_ints_casts.q.out b/ql/src/test/results/clientpositive/timestamp_ints_casts.q.out
index f687308..572c49e 100644
--- a/ql/src/test/results/clientpositive/timestamp_ints_casts.q.out
+++ b/ql/src/test/results/clientpositive/timestamp_ints_casts.q.out
@@ -49,17 +49,17 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cbigint % 250L) = 0L) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1684250 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1559690 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((cbigint % 250L) = 0L) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 842180 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 779900 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0L) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp),  [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-                Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 2641080 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6144 Data size: 2641080 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -187,17 +187,17 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cbigint % 250L) = 0L) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1684250 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1559690 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((cbigint % 250L) = 0L) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 842180 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 779900 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0L) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp),  [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-                Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 2641080 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6144 Data size: 2641080 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_aggregate_9.q.out b/ql/src/test/results/clientpositive/vector_aggregate_9.q.out
index 0aa46a2..0c823af 100644
--- a/ql/src/test/results/clientpositive/vector_aggregate_9.q.out
+++ b/ql/src/test/results/clientpositive/vector_aggregate_9.q.out
@@ -347,7 +347,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: vectortab2korc_n4
-            Statistics: Num rows: 2000 Data size: 80000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2000 Data size: 75760 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:t:tinyint, 1:si:smallint, 2:i:int, 3:b:bigint, 4:f:float, 5:d:double, 6:dc:decimal(38,18), 7:bo:boolean, 8:s:string, 9:s2:string, 10:ts:timestamp, 11:ts2:timestamp, 12:dt:date, 13:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -358,7 +358,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [10]
-              Statistics: Num rows: 2000 Data size: 80000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2000 Data size: 75760 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: min(ts), max(ts), sum(ts), count(ts)
                 Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/vector_coalesce.q.out b/ql/src/test/results/clientpositive/vector_coalesce.q.out
index 1ab2d6e..dd7df44 100644
--- a/ql/src/test/results/clientpositive/vector_coalesce.q.out
+++ b/ql/src/test/results/clientpositive/vector_coalesce.q.out
@@ -294,7 +294,7 @@ STAGE PLANS:
               Filter Vectorization:
                   className: VectorFilterOperator
                   native: true
-                  predicateExpression: FilterExprOrExpr(children: SelectColumnIsNotNull(col 8:timestamp), SelectColumnIsNotNull(col 9:timestamp))
+                  predicateExpression: FilterExprOrExpr(children: SelectColumnIsNotNull(col 9:timestamp), SelectColumnIsNotNull(col 8:timestamp))
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
diff --git a/ql/src/test/results/clientpositive/vector_decimal_cast.q.out b/ql/src/test/results/clientpositive/vector_decimal_cast.q.out
index 8889a74..2180e62 100644
--- a/ql/src/test/results/clientpositive/vector_decimal_cast.q.out
+++ b/ql/src/test/results/clientpositive/vector_decimal_cast.q.out
@@ -20,8 +20,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc
-            filterExpr: (cboolean1 is not null and cint is not null and cdouble is not null and ctimestamp1 is not null) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 638316 Basic stats: COMPLETE Column stats: COMPLETE
+            filterExpr: (cboolean1 is not null and cint is not null and ctimestamp1 is not null and cdouble is not null) (type: boolean)
+            Statistics: Num rows: 12288 Data size: 513756 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -29,9 +29,9 @@ STAGE PLANS:
               Filter Vectorization:
                   className: VectorFilterOperator
                   native: true
-                  predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 10:boolean), SelectColumnIsNotNull(col 2:int), SelectColumnIsNotNull(col 5:double), SelectColumnIsNotNull(col 8:timestamp))
-              predicate: (cboolean1 is not null and cint is not null and cdouble is not null and ctimestamp1 is not null) (type: boolean)
-              Statistics: Num rows: 5112 Data size: 265564 Basic stats: COMPLETE Column stats: COMPLETE
+                  predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 10:boolean), SelectColumnIsNotNull(col 2:int), SelectColumnIsNotNull(col 8:timestamp), SelectColumnIsNotNull(col 5:double))
+              predicate: (cboolean1 is not null and cint is not null and ctimestamp1 is not null and cdouble is not null) (type: boolean)
+              Statistics: Num rows: 3816 Data size: 159600 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cdouble (type: double), cint (type: int), cboolean1 (type: boolean), ctimestamp1 (type: timestamp), CAST( cdouble AS decimal(20,10)) (type: decimal(20,10)), CAST( cint AS decimal(23,14)) (type: decimal(23,14)), CAST( cboolean1 AS decimal(5,2)) (type: decimal(5,2)), CAST( ctimestamp1 AS decimal(15,0)) (type: decimal(15,0))
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
@@ -40,19 +40,19 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [5, 2, 10, 8, 13, 14, 15, 16]
                     selectExpressions: CastDoubleToDecimal(col 5:double) -> 13:decimal(20,10), CastLongToDecimal(col 2:int) -> 14:decimal(23,14), CastLongToDecimal(col 10:boolean) -> 15:decimal(5,2), CastTimestampToDecimal(col 8:timestamp) -> 16:decimal(15,0)
-                Statistics: Num rows: 5112 Data size: 2410700 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3816 Data size: 1760976 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 10
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 10 Data size: 4784 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 10 Data size: 4704 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 10 Data size: 4784 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 10 Data size: 4704 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -144,8 +144,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypes_small
-            filterExpr: (cboolean1 is not null and cint is not null and cdouble is not null and ctimestamp1 is not null) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 638316 Basic stats: COMPLETE Column stats: COMPLETE
+            filterExpr: (cboolean1 is not null and cint is not null and ctimestamp1 is not null and cdouble is not null) (type: boolean)
+            Statistics: Num rows: 12288 Data size: 513756 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -153,9 +153,9 @@ STAGE PLANS:
               Filter Vectorization:
                   className: VectorFilterOperator
                   native: true
-                  predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 10:boolean), SelectColumnIsNotNull(col 2:int), SelectColumnIsNotNull(col 5:double), SelectColumnIsNotNull(col 8:timestamp))
-              predicate: (cboolean1 is not null and cint is not null and cdouble is not null and ctimestamp1 is not null) (type: boolean)
-              Statistics: Num rows: 5112 Data size: 265564 Basic stats: COMPLETE Column stats: COMPLETE
+                  predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 10:boolean), SelectColumnIsNotNull(col 2:int), SelectColumnIsNotNull(col 8:timestamp), SelectColumnIsNotNull(col 5:double))
+              predicate: (cboolean1 is not null and cint is not null and ctimestamp1 is not null and cdouble is not null) (type: boolean)
+              Statistics: Num rows: 3816 Data size: 159600 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cdouble (type: double), cint (type: int), cboolean1 (type: boolean), ctimestamp1 (type: timestamp), CAST( cdouble AS decimal(20,10)) (type: decimal(20,10)), CAST( cint AS decimal(23,14)) (type: decimal(23,14)), CAST( cboolean1 AS decimal(5,2)) (type: decimal(5,2)), CAST( ctimestamp1 AS decimal(15,0)) (type: decimal(15,0))
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
@@ -164,19 +164,19 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [5, 2, 10, 8, 13, 14, 15, 16]
                     selectExpressions: CastDoubleToDecimal(col 5:double) -> 13:decimal(20,10), CastLongToDecimal(col 2:int) -> 14:decimal(23,14), CastLongToDecimal(col 10:boolean) -> 15:decimal(5,2), CastTimestampToDecimal(col 8:timestamp) -> 16:decimal(15,0)
-                Statistics: Num rows: 5112 Data size: 2410700 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3816 Data size: 1760976 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 10
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 10 Data size: 4784 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 10 Data size: 4704 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 10 Data size: 4784 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 10 Data size: 4704 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_empty_where.q.out b/ql/src/test/results/clientpositive/vector_empty_where.q.out
index 713357f..61f6d80 100644
--- a/ql/src/test/results/clientpositive/vector_empty_where.q.out
+++ b/ql/src/test/results/clientpositive/vector_empty_where.q.out
@@ -483,7 +483,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc
-            Statistics: Num rows: 12288 Data size: 528216 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 403656 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -492,7 +492,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: SelectColumnIsTrue(col 13:boolean)(children: CastTimestampToBoolean(col 8:timestamp) -> 13:boolean)
               predicate: ctimestamp1 (type: timestamp)
-              Statistics: Num rows: 6144 Data size: 264108 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 201828 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cint (type: int)
                 outputColumnNames: cint
@@ -500,7 +500,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [2]
-                Statistics: Num rows: 6144 Data size: 264108 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 201828 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   Group By Vectorization:
                       className: VectorGroupByOperator
diff --git a/ql/src/test/results/clientpositive/vector_interval_mapjoin.q.out b/ql/src/test/results/clientpositive/vector_interval_mapjoin.q.out
index c5584e7..e382b73 100644
--- a/ql/src/test/results/clientpositive/vector_interval_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/vector_interval_mapjoin.q.out
@@ -207,14 +207,14 @@ STAGE PLANS:
           TableScan
             alias: vectortab_b_1korc
             filterExpr: (dt is not null and CAST( ts AS DATE) is not null and s is not null) (type: boolean)
-            Statistics: Num rows: 1000 Data size: 186864 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 184664 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (dt is not null and CAST( ts AS DATE) is not null and s is not null) (type: boolean)
-              Statistics: Num rows: 943 Data size: 176202 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 890 Data size: 164340 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: s (type: string), (dt - CAST( ts AS DATE)) (type: interval_day_time)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 943 Data size: 99958 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 890 Data size: 94340 Basic stats: COMPLETE Column stats: COMPLETE
                 HashTable Sink Operator
                   keys:
                     0 _col0 (type: string), _col1 (type: interval_day_time)
@@ -226,7 +226,7 @@ STAGE PLANS:
           TableScan
             alias: vectortab_a_1korc
             filterExpr: (dt is not null and CAST( ts AS DATE) is not null and s is not null) (type: boolean)
-            Statistics: Num rows: 1000 Data size: 187480 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 185480 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -235,7 +235,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 12:date), SelectColumnIsNotNull(col 14:date)(children: CastTimestampToDate(col 10:timestamp) -> 14:date), SelectColumnIsNotNull(col 8:string))
               predicate: (dt is not null and CAST( ts AS DATE) is not null and s is not null) (type: boolean)
-              Statistics: Num rows: 954 Data size: 178852 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 905 Data size: 167854 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: s (type: string), (dt - CAST( ts AS DATE)) (type: interval_day_time)
                 outputColumnNames: _col0, _col1
@@ -244,7 +244,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [8, 16]
                     selectExpressions: DateColSubtractDateColumn(col 12:date, col 15:date)(children: CastTimestampToDate(col 10:timestamp) -> 15:date) -> 16:interval_day_time
-                Statistics: Num rows: 954 Data size: 101124 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 905 Data size: 95930 Basic stats: COMPLETE Column stats: COMPLETE
                 Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -259,7 +259,7 @@ STAGE PLANS:
                       nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true
                       nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 34600 Data size: 6920000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 32218 Data size: 6443600 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string), _col2 (type: string), _col1 (type: interval_day_time)
                     outputColumnNames: _col0, _col1, _col2
@@ -267,13 +267,13 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [0, 2, 1]
-                    Statistics: Num rows: 34600 Data size: 6920000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 32218 Data size: 6443600 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
                       File Sink Vectorization:
                           className: VectorFileSinkOperator
                           native: false
-                      Statistics: Num rows: 34600 Data size: 6920000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 32218 Data size: 6443600 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_non_constant_in_expr.q.out b/ql/src/test/results/clientpositive/vector_non_constant_in_expr.q.out
index d99cf4d..58fa5b3 100644
--- a/ql/src/test/results/clientpositive/vector_non_constant_in_expr.q.out
+++ b/ql/src/test/results/clientpositive/vector_non_constant_in_expr.q.out
@@ -21,17 +21,17 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cint = UDFToInteger(ctinyint)) or (UDFToLong(cint) = cbigint)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((cint = UDFToInteger(ctinyint)) or (UDFToLong(cint) = cbigint)) (type: boolean)
-              Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2844090 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_outer_join1.q.out b/ql/src/test/results/clientpositive/vector_outer_join1.q.out
index b48ff79..aa6bffb 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join1.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join1.q.out
@@ -253,14 +253,14 @@ STAGE PLANS:
           TableScan
             alias: cd
             filterExpr: cint is not null (type: boolean)
-            Statistics: Num rows: 15 Data size: 3745 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 15 Data size: 3545 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: cint is not null (type: boolean)
-              Statistics: Num rows: 10 Data size: 2640 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 10 Data size: 2520 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 10 Data size: 2640 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 10 Data size: 2520 Basic stats: COMPLETE Column stats: COMPLETE
                 HashTable Sink Operator
                   keys:
                     0 _col2 (type: int)
@@ -271,7 +271,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: c
-            Statistics: Num rows: 15 Data size: 3745 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 15 Data size: 3545 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -282,7 +282,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]
-              Statistics: Num rows: 15 Data size: 3745 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 15 Data size: 3545 Basic stats: COMPLETE Column stats: COMPLETE
               Map Join Operator
                 condition map:
                      Left Outer Join 0 to 1
@@ -297,13 +297,13 @@ STAGE PLANS:
                     nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Outer Join has keys IS true, Optimized Table and Supports Key Types IS true
                     nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
-                Statistics: Num rows: 33 Data size: 14459 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 33 Data size: 14099 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 33 Data size: 14459 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 33 Data size: 14099 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_outer_join3.q.out b/ql/src/test/results/clientpositive/vector_outer_join3.q.out
index 43d9ddf..ccf3d9a 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join3.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join3.q.out
@@ -248,7 +248,7 @@ left outer join small_alltypesorc_a_n1 hd
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@small_alltypesorc_a_n1
 #### A masked pattern was here ####
-{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_a_n1\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 20.0,\n      \"avgRowSize\": 10.0,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\"\n        } [...]
+{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_a_n1\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 20.0,\n      \"avgRowSize\": 10.0,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\"\n        } [...]
 PREHOOK: query: select count(*) from (select c.cstring1
 from small_alltypesorc_a_n1 c
 left outer join small_alltypesorc_a_n1 cd
@@ -292,7 +292,7 @@ left outer join small_alltypesorc_a_n1 hd
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@small_alltypesorc_a_n1
 #### A masked pattern was here ####
-{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_a_n1\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 20.0,\n      \"avgRowSize\": 14.75,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\"\n         [...]
+{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_a_n1\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 20.0,\n      \"avgRowSize\": 14.75,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\"\n         [...]
 PREHOOK: query: select count(*) from (select c.cstring1
 from small_alltypesorc_a_n1 c
 left outer join small_alltypesorc_a_n1 cd
@@ -336,7 +336,7 @@ left outer join small_alltypesorc_a_n1 hd
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@small_alltypesorc_a_n1
 #### A masked pattern was here ####
-{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_a_n1\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 20.0,\n      \"avgRowSize\": 26.75,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\"\n         [...]
+{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_a_n1\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 20.0,\n      \"avgRowSize\": 26.75,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\"\n         [...]
 PREHOOK: query: select count(*) from (select c.cstring1
 from small_alltypesorc_a_n1 c
 left outer join small_alltypesorc_a_n1 cd
diff --git a/ql/src/test/results/clientpositive/vector_outer_join4.q.out b/ql/src/test/results/clientpositive/vector_outer_join4.q.out
index 6686575..0ddb8f1 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join4.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join4.q.out
@@ -262,7 +262,7 @@ left outer join small_alltypesorc_b cd
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@small_alltypesorc_b
 #### A masked pattern was here ####
-{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_b\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 30.0,\n      \"avgRowSize\": 139.86666666666667,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\" [...]
+{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_b\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 30.0,\n      \"avgRowSize\": 139.86666666666667,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\" [...]
 PREHOOK: query: select * 
 from small_alltypesorc_b c
 left outer join small_alltypesorc_b cd
@@ -347,7 +347,7 @@ left outer join small_alltypesorc_b hd
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@small_alltypesorc_b
 #### A masked pattern was here ####
-{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_b\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 30.0,\n      \"avgRowSize\": 4.0,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\"\n        },\n  [...]
+{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_b\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 30.0,\n      \"avgRowSize\": 4.0,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\"\n        },\n  [...]
 PREHOOK: query: select c.ctinyint 
 from small_alltypesorc_b c
 left outer join small_alltypesorc_b hd
@@ -794,7 +794,7 @@ left outer join small_alltypesorc_b hd
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@small_alltypesorc_b
 #### A masked pattern was here ####
-{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_b\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 30.0,\n      \"avgRowSize\": 8.0,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\"\n        },\n  [...]
+{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"small_alltypesorc_b\"\n      ],\n      \"table:alias\": \"c\",\n      \"inputs\": [],\n      \"rowCount\": 30.0,\n      \"avgRowSize\": 8.0,\n      \"rowType\": [\n        {\n          \"type\": \"TINYINT\",\n          \"nullable\": true,\n          \"name\": \"ctinyint\"\n        },\n  [...]
 PREHOOK: query: select count(*) from (select c.ctinyint
 from small_alltypesorc_b c
 left outer join small_alltypesorc_b cd
diff --git a/ql/src/test/results/clientpositive/vector_udf_trunc.q.out b/ql/src/test/results/clientpositive/vector_udf_trunc.q.out
index a6ce91f..2220937 100644
--- a/ql/src/test/results/clientpositive/vector_udf_trunc.q.out
+++ b/ql/src/test/results/clientpositive/vector_udf_trunc.q.out
@@ -97,7 +97,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc
-            Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 366960 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -219,7 +219,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc
-            Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 366960 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -341,7 +341,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc
-            Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 366960 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
diff --git a/ql/src/test/results/clientpositive/vectorization_10.q.out b/ql/src/test/results/clientpositive/vectorization_10.q.out
index d13536b..601e401 100644
--- a/ql/src/test/results/clientpositive/vectorization_10.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_10.q.out
@@ -65,7 +65,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cstring2 <= '10') or ((UDFToDouble(ctinyint) > cdouble) and (CAST( ctinyint AS decimal(6,2)) <= -5638.15)) or ((cdouble > 6981.0D) and ((CAST( csmallint AS decimal(11,4)) = 9763215.5639) or (cstring1 like '%a')))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -75,7 +75,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterStringGroupColLessEqualStringScalar(col 7:string, val 10), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleColumn(col 13:double, col 5:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterDecimalColLessEqualDecimalScalar(col 14:decimal(6,2), val -5638.15)(children: CastLongToDecimal(col 0:tinyint) -> 14:decimal(6,2))), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 5:doub [...]
               predicate: ((cstring2 <= '10') or ((UDFToDouble(ctinyint) > cdouble) and (CAST( ctinyint AS decimal(6,2)) <= -5638.15)) or ((cdouble > 6981.0D) and ((CAST( csmallint AS decimal(11,4)) = 9763215.5639) or (cstring1 like '%a')))) (type: boolean)
-              Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cdouble (type: double), ctimestamp1 (type: timestamp), ctinyint (type: tinyint), cboolean1 (type: boolean), cstring1 (type: string), (- cdouble) (type: double), (cdouble + UDFToDouble(csmallint)) (type: double), ((cdouble + UDFToDouble(csmallint)) % 33.0D) (type: double), (- cdouble) (type: double), (UDFToDouble(ctinyint) % cdouble) (type: double), (UDFToShort(ctinyint) % csmallint) (type: smallint), (- cdouble) (type: double), (cbigint * UDFToLong((UDFToShor [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -84,13 +84,13 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [5, 8, 0, 10, 6, 16, 18, 21, 22, 24, 25, 26, 28, 31, 33]
                     selectExpressions: DoubleColUnaryMinus(col 5:double) -> 16:double, DoubleColAddDoubleColumn(col 5:double, col 17:double)(children: CastLongToDouble(col 1:smallint) -> 17:double) -> 18:double, DoubleColModuloDoubleScalar(col 20:double, val 33.0)(children: DoubleColAddDoubleColumn(col 5:double, col 19:double)(children: CastLongToDouble(col 1:smallint) -> 19:double) -> 20:double) -> 21:double, DoubleColUnaryMinus(col 5:double) -> 22:double, DoubleColModuloDoubleColumn(co [...]
-                Statistics: Num rows: 12288 Data size: 2434654 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12288 Data size: 2310094 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 12288 Data size: 2434654 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2310094 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_11.q.out b/ql/src/test/results/clientpositive/vectorization_11.q.out
index 1040d3a..1497854 100644
--- a/ql/src/test/results/clientpositive/vectorization_11.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_11.q.out
@@ -47,7 +47,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((ctimestamp1 is null and (cstring1 like '%a')) or (cstring2 = cstring1)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2381474 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2256914 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -57,7 +57,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: SelectColumnIsNull(col 8:timestamp), FilterStringColLikeStringScalar(col 6:string, pattern %a)), FilterStringGroupColEqualStringGroupColumn(col 7:string, col 6:string))
               predicate: ((ctimestamp1 is null and (cstring1 like '%a')) or (cstring2 = cstring1)) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 1190792 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 7701 Data size: 1414500 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cstring1 (type: string), cboolean1 (type: boolean), cdouble (type: double), ctimestamp1 (type: timestamp), (-3728 * UDFToInteger(csmallint)) (type: int), (cdouble - 9763215.5639D) (type: double), (- cdouble) (type: double), ((- cdouble) + 6981.0D) (type: double), (cdouble * -5638.15D) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -66,13 +66,13 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [6, 10, 5, 8, 13, 14, 15, 17, 18]
                     selectExpressions: LongScalarMultiplyLongColumn(val -3728, col 1:int)(children: col 1:smallint) -> 13:int, DoubleColSubtractDoubleScalar(col 5:double, val 9763215.5639) -> 14:double, DoubleColUnaryMinus(col 5:double) -> 15:double, DoubleColAddDoubleScalar(col 16:double, val 6981.0)(children: DoubleColUnaryMinus(col 5:double) -> 16:double) -> 17:double, DoubleColMultiplyDoubleScalar(col 5:double, val -5638.15) -> 18:double
-                Statistics: Num rows: 6144 Data size: 953272 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7701 Data size: 1116736 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 6144 Data size: 953272 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 7701 Data size: 1116736 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_12.q.out b/ql/src/test/results/clientpositive/vectorization_12.q.out
index 977372a..c00a6fd 100644
--- a/ql/src/test/results/clientpositive/vectorization_12.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_12.q.out
@@ -82,7 +82,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: (((cstring1 like '%a') or ((cboolean2 <= 1) and (cbigint >= UDFToLong(csmallint)))) and ((cboolean1 >= cboolean2) or (UDFToShort(ctinyint) <> csmallint)) and ctimestamp1 is null) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1647554 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1522994 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -92,7 +92,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterStringColLikeStringScalar(col 6:string, pattern %a), FilterExprAndExpr(children: FilterLongColLessEqualLongScalar(col 11:boolean, val 1), FilterLongColGreaterEqualLongColumn(col 3:bigint, col 1:bigint)(children: col 1:smallint))), FilterExprOrExpr(children: FilterLongColGreaterEqualLongColumn(col 10:boolean, col 11:boolean), FilterLongColNotEqualLongColumn(col 0:smallint, col 1:smallint)(c [...]
               predicate: (((cstring1 like '%a') or ((cboolean2 <= 1) and (cbigint >= UDFToLong(csmallint)))) and ((cboolean1 >= cboolean2) or (UDFToShort(ctinyint) <> csmallint)) and ctimestamp1 is null) (type: boolean)
-              Statistics: Num rows: 1 Data size: 166 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1903 Data size: 236052 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cbigint (type: bigint), cboolean1 (type: boolean), cstring1 (type: string), cdouble (type: double), UDFToDouble(cbigint) (type: double), (UDFToDouble(cbigint) * UDFToDouble(cbigint)) (type: double), (cdouble * cdouble) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
@@ -101,7 +101,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [3, 10, 6, 5, 13, 16, 17]
                     selectExpressions: CastLongToDouble(col 3:bigint) -> 13:double, DoubleColMultiplyDoubleColumn(col 14:double, col 15:double)(children: CastLongToDouble(col 3:bigint) -> 14:double, CastLongToDouble(col 3:bigint) -> 15:double) -> 16:double, DoubleColMultiplyDoubleColumn(col 5:double, col 5:double) -> 17:double
-                Statistics: Num rows: 1 Data size: 166 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1903 Data size: 236052 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(_col0), sum(_col5), sum(_col4), sum(_col3), count(_col3), sum(_col0), sum(_col6)
                   Group By Vectorization:
@@ -116,7 +116,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-                  Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1903 Data size: 268676 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: double), _col1 (type: bigint), _col2 (type: string), _col3 (type: boolean)
                     sort order: ++++
@@ -126,7 +126,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: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1903 Data size: 268676 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col4 (type: bigint), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: bigint), _col9 (type: bigint), _col10 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -154,11 +154,11 @@ STAGE PLANS:
           keys: KEY._col0 (type: double), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: boolean)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
-          Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1903 Data size: 268676 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col1 (type: bigint), _col3 (type: boolean), _col2 (type: string), _col0 (type: double), (-6432.0D * _col0) (type: double), (- _col1) (type: bigint), _col4 (type: bigint), (_col1 * _col4) (type: bigint), power(((_col5 - ((_col6 * _col6) / _col4)) / CASE WHEN ((_col4 = 1L)) THEN (null) ELSE ((_col4 - 1)) END), 0.5) (type: double), ((-6432.0D * _col0) / -6432.0D) (type: double), (- ((-6432.0D * _col0) / -6432.0D)) (type: double), (_col7 / _col8) (type: double), (-  [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col17, _col18, _col19
-            Statistics: Num rows: 1 Data size: 346 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1903 Data size: 603604 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -181,7 +181,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: 1 Data size: 346 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1903 Data size: 603604 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col1 (type: boolean), _col4 (type: double), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: double), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: double), _col13 (type: decimal(22,2)), _col14 (type: bigint), _col15 (type: double), _col17 (type: double), _col18 (type: double), _col19 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -207,10 +207,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey1 (type: bigint), VALUE._col0 (type: boolean), KEY.reducesinkkey2 (type: string), null (type: timestamp), KEY.reducesinkkey0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: bigint), VALUE._col3 (type: bigint), VALUE._col4 (type: bigint), VALUE._col5 (type: double), VALUE._col6 (type: double), VALUE._col7 (type: double), VALUE._col8 (type: double), VALUE._col9 (type: double), VALUE._col10 (type: decimal(22,2)), VALUE._col11 (type: big [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-          Statistics: Num rows: 1 Data size: 386 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1903 Data size: 603644 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 386 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1903 Data size: 603644 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_13.q.out b/ql/src/test/results/clientpositive/vectorization_13.q.out
index 3552007..cc01a73 100644
--- a/ql/src/test/results/clientpositive/vectorization_13.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_13.q.out
@@ -84,7 +84,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28789.0D) and (UDFToDouble(ctimestamp2) <> -28788.0D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2028982 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1779902 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -94,7 +94,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val 3569.0), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 10.175), FilterLongColNotEqualLongScalar(col 10:boolean, val 1)), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28789.0)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val -28788.0 [...]
               predicate: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28789.0D) and (UDFToDouble(ctimestamp2) <> -28788.0D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-              Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -103,7 +103,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [10, 0, 8, 4, 6, 4, 16, 17, 20]
                     selectExpressions: DoubleColMultiplyDoubleColumn(col 4:double, col 4:double)(children: col 4:float, col 4:float) -> 16:double, CastLongToDouble(col 0:tinyint) -> 17:double, DoubleColMultiplyDoubleColumn(col 18:double, col 19:double)(children: CastLongToDouble(col 0:tinyint) -> 18:double, CastLongToDouble(col 0:tinyint) -> 19:double) -> 20:double
-                Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: max(_col1), sum(_col3), sum(_col6), sum(_col5), count(_col3), sum(_col8), sum(_col7), count(_col1), max(_col3), min(_col1)
                   Group By Vectorization:
@@ -118,7 +118,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-                  Statistics: Num rows: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
                     sort order: +++++
@@ -128,7 +128,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: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: double), _col8 (type: double), _col9 (type: bigint), _col10 (type: double), _col11 (type: double), _col12 (type: bigint), _col13 (type: float), _col14 (type: tinyint)
       Execution mode: vectorized
       Map Vectorization:
@@ -156,11 +156,11 @@ STAGE PLANS:
           keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-          Statistics: Num rows: 346 Data size: 64822 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * _col3) (type: float), power(((_col7 - ((_col8 * _col8) / _col9)) / _col9), 0.5) (type: double), (- _col6) (type: double), power(((_col10 - ((_col11 * [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-            Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -183,7 +183,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: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
               TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -209,13 +209,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: boolean), KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey2 (type: timestamp), KEY.reducesinkkey3 (type: float), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: tinyint), KEY.reducesinkkey6 (type: tinyint), KEY.reducesinkkey7 (type: tinyint), KEY.reducesinkkey8 (type: double), KEY.reducesinkkey9 (type: double), KEY.reducesinkkey10 (type: double), KEY.reducesinkkey11 (type: float), KEY.reducesinkkey12 (type: double), KEY.r [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-          Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 40
-            Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 40 Data size: 12846 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 40 Data size: 12846 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -421,7 +421,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28801.388D) and (UDFToDouble(ctimestamp2) <> -28801.336D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2028982 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1779902 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -430,7 +430,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val 3569.0), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 10.175), FilterLongColNotEqualLongScalar(col 10:boolean, val 1)), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28801.388)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val -28801 [...]
               predicate: (((cfloat < 3569.0) and (cdouble <= 10.175D) and (cboolean1 <> 1)) or ((UDFToDouble(ctimestamp1) > -28801.388D) and (UDFToDouble(ctimestamp2) <> -28801.336D) and (CAST( ctinyint AS decimal(11,4)) < 9763215.5639))) (type: boolean)
-              Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -439,7 +439,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [10, 0, 8, 4, 6, 4, 16, 17, 20]
                     selectExpressions: DoubleColMultiplyDoubleColumn(col 4:double, col 4:double)(children: col 4:float, col 4:float) -> 16:double, CastLongToDouble(col 0:tinyint) -> 17:double, DoubleColMultiplyDoubleColumn(col 18:double, col 19:double)(children: CastLongToDouble(col 0:tinyint) -> 18:double, CastLongToDouble(col 0:tinyint) -> 19:double) -> 20:double
-                Statistics: Num rows: 1386 Data size: 228984 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1386 Data size: 200984 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: max(_col1), sum(_col3), sum(_col6), sum(_col5), count(_col3), sum(_col8), sum(_col7), count(_col1), max(_col3), min(_col1)
                   Group By Vectorization:
@@ -454,7 +454,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-                  Statistics: Num rows: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string)
                     sort order: +++++
@@ -464,7 +464,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: 693 Data size: 129752 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: double), _col8 (type: double), _col9 (type: bigint), _col10 (type: double), _col11 (type: double), _col12 (type: bigint), _col13 (type: float), _col14 (type: tinyint)
       Execution mode: vectorized
       Map Vectorization:
@@ -486,11 +486,11 @@ STAGE PLANS:
           keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-          Statistics: Num rows: 346 Data size: 64822 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 693 Data size: 122752 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * UDFToDouble(((- _col1) + _col5))) (type: double), (- _col6) (type: double), (79.553 * _col3) (type: float), power(((_col7 - ((_col8 * _col8) / _col9)) / _col9), 0.5) (type: double), (- _col6) (type: double), power(((_col10 - ((_col11 * [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-            Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -512,7 +512,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: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
               TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -532,13 +532,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: boolean), KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey2 (type: timestamp), KEY.reducesinkkey3 (type: float), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: tinyint), KEY.reducesinkkey6 (type: tinyint), KEY.reducesinkkey7 (type: tinyint), KEY.reducesinkkey8 (type: double), KEY.reducesinkkey9 (type: double), KEY.reducesinkkey10 (type: double), KEY.reducesinkkey11 (type: float), KEY.reducesinkkey12 (type: double), KEY.r [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-          Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 693 Data size: 219772 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 40
-            Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 40 Data size: 12846 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 40 Data size: 12846 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_14.q.out b/ql/src/test/results/clientpositive/vectorization_14.q.out
index c73cf60..72e7e3b 100644
--- a/ql/src/test/results/clientpositive/vectorization_14.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_14.q.out
@@ -84,7 +84,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((UDFToLong(ctinyint) <= cbigint) and (cdouble < UDFToDouble(ctinyint)) and ((cbigint > -257L) or (cfloat < UDFToFloat(cint))) and ((UDFToDouble(cint) <= cdouble) or (ctimestamp2 < ctimestamp1))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2139070 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1889990 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -94,7 +94,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterLongColLessEqualLongColumn(col 0:bigint, col 3:bigint)(children: col 0:tinyint), FilterDoubleColLessDoubleColumn(col 5:double, col 13:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterExprOrExpr(children: FilterLongColGreaterLongScalar(col 3:bigint, val -257), FilterDoubleColLessDoubleColumn(col 4:float, col 14:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 14:float)), FilterExprO [...]
               predicate: ((UDFToLong(ctinyint) <= cbigint) and (cdouble < UDFToDouble(ctinyint)) and ((cbigint > -257L) or (cfloat < UDFToFloat(cint))) and ((UDFToDouble(cint) <= cdouble) or (ctimestamp2 < ctimestamp1))) (type: boolean)
-              Statistics: Num rows: 758 Data size: 132082 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 758 Data size: 116802 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string), cboolean1 (type: boolean), cdouble (type: double), (- (-26.28D + cdouble)) (type: double), ((- (-26.28D + cdouble)) * (- (-26.28D + cdouble))) (type: double), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -103,7 +103,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [8, 4, 6, 10, 5, 17, 22, 4, 23]
                     selectExpressions: DoubleColUnaryMinus(col 16:double)(children: DoubleScalarAddDoubleColumn(val -26.28, col 5:double) -> 16:double) -> 17:double, DoubleColMultiplyDoubleColumn(col 19:double, col 21:double)(children: DoubleColUnaryMinus(col 18:double)(children: DoubleScalarAddDoubleColumn(val -26.28, col 5:double) -> 18:double) -> 19:double, DoubleColUnaryMinus(col 20:double)(children: DoubleScalarAddDoubleColumn(val -26.28, col 5:double) -> 20:double) -> 21:double) -> [...]
-                Statistics: Num rows: 758 Data size: 132082 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 758 Data size: 116802 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col6), sum(_col5), count(_col5), max(_col1), sum(_col8), sum(_col7), count(_col1)
                   Group By Vectorization:
@@ -118,7 +118,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                  Statistics: Num rows: 379 Data size: 66108 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 379 Data size: 62308 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: float), _col2 (type: double), _col3 (type: timestamp), _col4 (type: boolean)
                     sort order: +++++
@@ -128,7 +128,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: 379 Data size: 66108 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 379 Data size: 62308 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col5 (type: double), _col6 (type: double), _col7 (type: bigint), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: bigint)
       Execution mode: vectorized
       Map Vectorization:
@@ -156,11 +156,11 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: float), KEY._col2 (type: double), KEY._col3 (type: timestamp), KEY._col4 (type: boolean)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-          Statistics: Num rows: 189 Data size: 33008 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 379 Data size: 62308 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col3 (type: timestamp), _col1 (type: float), _col0 (type: string), _col4 (type: boolean), _col2 (type: double), (-26.28D + _col2) (type: double), (- (-26.28D + _col2)) (type: double), power(((_col5 - ((_col6 * _col6) / _col7)) / CASE WHEN ((_col7 = 1L)) THEN (null) ELSE ((_col7 - 1)) END), 0.5) (type: double), (_col1 * -26.28) (type: float), _col8 (type: float), (- _col1) (type: float), (- _col8) (type: float), ((- (-26.28D + _col2)) / 10.175D) (type: double), p [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21
-            Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 379 Data size: 88080 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -183,7 +183,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: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 379 Data size: 88080 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col3 (type: boolean), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: float), _col10 (type: float), _col11 (type: float), _col12 (type: double), _col13 (type: double), _col14 (type: bigint), _col15 (type: double), _col16 (type: double), _col17 (type: double), _col18 (type: double), _col19 (type: double), _col20 (type: double), _col21 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -209,10 +209,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey3 (type: timestamp), KEY.reducesinkkey1 (type: float), KEY.reducesinkkey0 (type: string), VALUE._col0 (type: boolean), KEY.reducesinkkey2 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double), VALUE._col3 (type: double), VALUE._col4 (type: float), VALUE._col5 (type: float), VALUE._col6 (type: float), VALUE._col7 (type: float), VALUE._col8 (type: double), VALUE._col9 (type: double), VALUE._col10 (type: bigint), VALUE._col11 (type: d [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21
-          Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 379 Data size: 88080 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 379 Data size: 88080 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_15.q.out b/ql/src/test/results/clientpositive/vectorization_15.q.out
index 1a5de65..dbfa0d8 100644
--- a/ql/src/test/results/clientpositive/vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_15.q.out
@@ -80,7 +80,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cstring1 like '10%') or (cstring2 like '%ss%') or ((cint >= -75) and (UDFToShort(ctinyint) = csmallint) and (cdouble >= -3728.0D))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -90,7 +90,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterStringColLikeStringScalar(col 6:string, pattern 10%), FilterStringColLikeStringScalar(col 7:string, pattern %ss%), FilterExprAndExpr(children: FilterLongColGreaterEqualLongScalar(col 2:int, val -75), FilterLongColEqualLongColumn(col 0:smallint, col 1:smallint)(children: col 0:tinyint), FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -3728.0)))
               predicate: ((cstring1 like '10%') or (cstring2 like '%ss%') or ((cint >= -75) and (UDFToShort(ctinyint) = csmallint) and (cdouble >= -3728.0D))) (type: boolean)
-              Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cfloat (type: float), cboolean1 (type: boolean), cdouble (type: double), cstring1 (type: string), ctinyint (type: tinyint), cint (type: int), ctimestamp1 (type: timestamp), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble(cint)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -99,7 +99,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [4, 10, 5, 6, 0, 2, 8, 4, 13, 14, 17, 18, 21]
                     selectExpressions: DoubleColMultiplyDoubleColumn(col 4:double, col 4:double)(children: col 4:float, col 4:float) -> 13:double, CastLongToDouble(col 0:tinyint) -> 14:double, DoubleColMultiplyDoubleColumn(col 15:double, col 16:double)(children: CastLongToDouble(col 0:tinyint) -> 15:double, CastLongToDouble(col 0:tinyint) -> 16:double) -> 17:double, CastLongToDouble(col 2:int) -> 18:double, DoubleColMultiplyDoubleColumn(col 19:double, col 20:double)(children: CastLongToD [...]
-                Statistics: Num rows: 12288 Data size: 2491562 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12288 Data size: 2367002 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col8), sum(_col7), count(_col0), min(_col2), sum(_col10), sum(_col9), count(_col4), sum(_col12), sum(_col11), count(_col5)
                   Group By Vectorization:
@@ -114,7 +114,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
-                  Statistics: Num rows: 6144 Data size: 1278652 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6144 Data size: 1216372 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp)
                     sort order: +++++++
@@ -124,7 +124,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: 6144 Data size: 1278652 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6144 Data size: 1216372 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col7 (type: double), _col8 (type: double), _col9 (type: bigint), _col10 (type: double), _col11 (type: double), _col12 (type: double), _col13 (type: bigint), _col14 (type: double), _col15 (type: double), _col16 (type: bigint)
       Execution mode: vectorized
       Map Vectorization:
@@ -151,11 +151,11 @@ STAGE PLANS:
           keys: KEY._col0 (type: float), KEY._col1 (type: boolean), KEY._col2 (type: double), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int), KEY._col6 (type: timestamp)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
-          Statistics: Num rows: 3072 Data size: 639332 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6144 Data size: 1216372 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: float), _col1 (type: boolean), _col2 (type: double), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int), _col6 (type: timestamp), power(((_col7 - ((_col8 * _col8) / _col9)) / CASE WHEN ((_col9 = 1L)) THEN (null) ELSE ((_col9 - 1)) END), 0.5) (type: double), (-26.28 - CAST( _col5 AS decimal(10,0))) (type: decimal(13,2)), _col10 (type: double), (_col2 * 79.553D) (type: double), (33.0 % _col0) (type: float), power(((_col11 - ((_col12 * _col1 [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-            Statistics: Num rows: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6144 Data size: 2592628 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -178,7 +178,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: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 2592628 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col7 (type: double), _col8 (type: decimal(13,2)), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: double), _col13 (type: double), _col14 (type: double), _col15 (type: tinyint), _col16 (type: double), _col17 (type: float), _col18 (type: int), _col19 (type: decimal(13,2)), _col20 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -203,10 +203,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: float), KEY.reducesinkkey1 (type: boolean), KEY.reducesinkkey2 (type: double), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: tinyint), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey6 (type: timestamp), VALUE._col0 (type: double), VALUE._col1 (type: decimal(13,2)), VALUE._col2 (type: double), VALUE._col3 (type: double), VALUE._col4 (type: float), VALUE._col5 (type: double), VALUE._col6 (type: double), VALUE._col7 (type: do [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
-          Statistics: Num rows: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6144 Data size: 2592628 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 3072 Data size: 1327460 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6144 Data size: 2592628 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_16.q.out b/ql/src/test/results/clientpositive/vectorization_16.q.out
index 2dfcc77..9c782f2 100644
--- a/ql/src/test/results/clientpositive/vectorization_16.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_16.q.out
@@ -57,7 +57,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2308074 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2183514 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -67,7 +67,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterStringColLikeStringScalar(col 7:string, pattern %b%), FilterExprOrExpr(children: FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -1.389), FilterStringGroupColLessStringScalar(col 6:string, val a)))
               predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp), (cdouble * cdouble) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3
@@ -76,7 +76,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [6, 5, 8, 13]
                     selectExpressions: DoubleColMultiplyDoubleColumn(col 5:double, col 5:double) -> 13:double
-                Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(_col1), sum(_col3), sum(_col1), min(_col1)
                   Group By Vectorization:
@@ -91,7 +91,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                  Statistics: Num rows: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
                     sort order: +++
@@ -101,7 +101,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: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col3 (type: bigint), _col4 (type: double), _col5 (type: double), _col6 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -129,14 +129,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-          Statistics: Num rows: 1536 Data size: 227586 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639D) (type: double), (- (_col1 - 9763215.5639D)) (type: double), _col3 (type: bigint), power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5) (type: double), (- power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5)) (type: double), (power(((_col4 - ((_col5 * _col5 [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-            Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 3072 Data size: 890996 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3072 Data size: 890996 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_17.q.out b/ql/src/test/results/clientpositive/vectorization_17.q.out
index ea65db1..68fcd18 100644
--- a/ql/src/test/results/clientpositive/vectorization_17.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_17.q.out
@@ -65,7 +65,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cbigint > -23L) and ((ctinyint >= 33Y) or (UDFToLong(csmallint) >= cbigint) or (UDFToDouble(cfloat) = cdouble)) and ((cdouble <> 988888.0D) or (CAST( cint AS decimal(13,3)) > -863.257))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1647550 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1522990 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -75,7 +75,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterLongColGreaterLongScalar(col 3:bigint, val -23), FilterExprOrExpr(children: FilterLongColGreaterEqualLongScalar(col 0:tinyint, val 33), FilterLongColGreaterEqualLongColumn(col 1:bigint, col 3:bigint)(children: col 1:smallint), FilterDoubleColEqualDoubleColumn(col 4:double, col 5:double)(children: col 4:float)), FilterExprOrExpr(children: FilterDoubleColNotEqualDoubleScalar(col 5:double, val 988888.0), FilterDecimalC [...]
               predicate: ((cbigint > -23L) and ((ctinyint >= 33Y) or (UDFToLong(csmallint) >= cbigint) or (UDFToDouble(cfloat) = cdouble)) and ((cdouble <> 988888.0D) or (CAST( cint AS decimal(13,3)) > -863.257))) (type: boolean)
-              Statistics: Num rows: 6141 Data size: 823456 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6141 Data size: 761216 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cfloat (type: float), cstring1 (type: string), cint (type: int), ctimestamp1 (type: timestamp), cdouble (type: double), cbigint (type: bigint), (UDFToDouble(cfloat) / UDFToDouble(ctinyint)) (type: double), (UDFToLong(cint) % cbigint) (type: bigint), (- cdouble) (type: double), (cdouble + (UDFToDouble(cfloat) / UDFToDouble(ctinyint))) (type: double), (cdouble / UDFToDouble(cint)) (type: double), (- (- cdouble)) (type: double), (9763215.5639 % CAST( cbigint AS  [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -84,7 +84,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [4, 6, 2, 8, 5, 3, 15, 16, 17, 20, 22, 24, 26, 29]
                     selectExpressions: DoubleColDivideDoubleColumn(col 4:double, col 14:double)(children: col 4:float, CastLongToDouble(col 0:tinyint) -> 14:double) -> 15:double, LongColModuloLongColumn(col 2:bigint, col 3:bigint)(children: col 2:int) -> 16:bigint, DoubleColUnaryMinus(col 5:double) -> 17:double, DoubleColAddDoubleColumn(col 5:double, col 19:double)(children: DoubleColDivideDoubleColumn(col 4:double, col 18:double)(children: col 4:float, CastLongToDouble(col 0:tinyint) -> [...]
-                Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6141 Data size: 1756220 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col5 (type: bigint), _col0 (type: float)
                   sort order: ++
@@ -93,7 +93,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: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6141 Data size: 1756220 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: timestamp), _col4 (type: double), _col6 (type: double), _col7 (type: bigint), _col8 (type: double), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: decimal(11,4)), _col13 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -119,10 +119,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey1 (type: float), VALUE._col0 (type: string), VALUE._col1 (type: int), VALUE._col2 (type: timestamp), VALUE._col3 (type: double), KEY.reducesinkkey0 (type: bigint), VALUE._col4 (type: double), VALUE._col5 (type: bigint), VALUE._col6 (type: double), VALUE._col7 (type: double), VALUE._col8 (type: double), VALUE._col9 (type: double), VALUE._col10 (type: decimal(11,4)), VALUE._col11 (type: double)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
-          Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6141 Data size: 1756220 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6141 Data size: 1756220 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_2.q.out b/ql/src/test/results/clientpositive/vectorization_2.q.out
index a5527a4..cdb904c 100644
--- a/ql/src/test/results/clientpositive/vectorization_2.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_2.q.out
@@ -63,7 +63,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: (((cdouble < UDFToDouble(ctinyint)) and ((UDFToDouble(ctimestamp2) <> -10669.0D) or (cint < 359))) or ((ctimestamp1 < ctimestamp2) and (cstring2 like 'b%') and (cfloat <= -5638.15))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2157324 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1908244 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -73,7 +73,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessDoubleColumn(col 5:double, col 13:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterExprOrExpr(children: FilterDoubleColNotEqualDoubleScalar(col 14:double, val -10669.0)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), FilterLongColLessLongScalar(col 2:int, val 359))), FilterExprAndExpr(children: FilterTimestampColLessTimestampColumn(col 8:t [...]
               predicate: (((cdouble < UDFToDouble(ctinyint)) and ((UDFToDouble(ctimestamp2) <> -10669.0D) or (cint < 359))) or ((ctimestamp1 < ctimestamp2) and (cstring2 like 'b%') and (cfloat <= -5638.15))) (type: boolean)
-              Statistics: Num rows: 4096 Data size: 719232 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 4096 Data size: 636272 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: csmallint (type: smallint), cfloat (type: float), cbigint (type: bigint), ctinyint (type: tinyint), cdouble (type: double), UDFToDouble(cbigint) (type: double), (UDFToDouble(cbigint) * UDFToDouble(cbigint)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
@@ -82,7 +82,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [1, 4, 3, 0, 5, 15, 18]
                     selectExpressions: CastLongToDouble(col 3:bigint) -> 15:double, DoubleColMultiplyDoubleColumn(col 16:double, col 17:double)(children: CastLongToDouble(col 3:bigint) -> 16:double, CastLongToDouble(col 3:bigint) -> 17:double) -> 18:double
-                Statistics: Num rows: 4096 Data size: 719232 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4096 Data size: 636272 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col0), count(_col0), sum(_col1), sum(_col6), sum(_col5), count(_col2), count(), min(_col3), sum(_col4), count(_col4)
                   Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/vectorization_3.q.out b/ql/src/test/results/clientpositive/vectorization_3.q.out
index add0a8d..0a9971b 100644
--- a/ql/src/test/results/clientpositive/vectorization_3.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_3.q.out
@@ -68,7 +68,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: (((UDFToFloat(cint) <= cfloat) and (CAST( cbigint AS decimal(22,3)) <> 79.553) and (UDFToDouble(ctimestamp2) = -29071.0D)) or ((UDFToDouble(cbigint) > cdouble) and (CAST( csmallint AS decimal(8,3)) >= 79.553) and (ctimestamp1 > ctimestamp2))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1276620 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1027540 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -78,7 +78,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleColumn(col 13:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 13:float), FilterDecimalColNotEqualDecimalScalar(col 14:decimal(22,3), val 79.553)(children: CastLongToDecimal(col 3:bigint) -> 14:decimal(22,3)), FilterDoubleColEqualDoubleScalar(col 15:double, val -29071.0)(children: CastTimestampToDouble(col 9:timestamp) -> 15:double)), Filte [...]
               predicate: (((UDFToFloat(cint) <= cfloat) and (CAST( cbigint AS decimal(22,3)) <> 79.553) and (UDFToDouble(ctimestamp2) = -29071.0D)) or ((UDFToDouble(cbigint) > cdouble) and (CAST( csmallint AS decimal(8,3)) >= 79.553) and (ctimestamp1 > ctimestamp2))) (type: boolean)
-              Statistics: Num rows: 2503 Data size: 260060 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2503 Data size: 209380 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: csmallint (type: smallint), ctinyint (type: tinyint), cfloat (type: float), cint (type: int), UDFToDouble(csmallint) (type: double), (UDFToDouble(csmallint) * UDFToDouble(csmallint)) (type: double), UDFToDouble(ctinyint) (type: double), (UDFToDouble(ctinyint) * UDFToDouble(ctinyint)) (type: double), UDFToDouble(cfloat) (type: double), (UDFToDouble(cfloat) * UDFToDouble(cfloat)) (type: double), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
@@ -87,7 +87,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [1, 0, 4, 2, 18, 21, 22, 25, 4, 26, 27, 30]
                     selectExpressions: CastLongToDouble(col 1:smallint) -> 18:double, DoubleColMultiplyDoubleColumn(col 19:double, col 20:double)(children: CastLongToDouble(col 1:smallint) -> 19:double, CastLongToDouble(col 1:smallint) -> 20:double) -> 21:double, CastLongToDouble(col 0:tinyint) -> 22:double, DoubleColMultiplyDoubleColumn(col 23:double, col 24:double)(children: CastLongToDouble(col 0:tinyint) -> 23:double, CastLongToDouble(col 0:tinyint) -> 24:double) -> 25:double, Double [...]
-                Statistics: Num rows: 2503 Data size: 260060 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2503 Data size: 209380 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col5), sum(_col4), count(_col0), sum(_col7), sum(_col6), count(_col1), sum(_col9), sum(_col8), count(_col2), sum(_col2), sum(_col3), count(_col3), sum(_col11), sum(_col10)
                   Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/vectorization_5.q.out b/ql/src/test/results/clientpositive/vectorization_5.q.out
index 8a1d404..ef485cb 100644
--- a/ql/src/test/results/clientpositive/vectorization_5.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_5.q.out
@@ -57,7 +57,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: (((cstring1 like '%b%') and cboolean2 is not null) or ((UDFToDouble(ctinyint) = cdouble) and (cstring2 like 'a') and ctimestamp2 is not null)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2454862 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2330342 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -67,7 +67,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterStringColLikeStringScalar(col 6:string, pattern %b%), SelectColumnIsNotNull(col 11:boolean)), FilterExprAndExpr(children: FilterDoubleColEqualDoubleColumn(col 13:double, col 5:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double), FilterStringColLikeStringScalar(col 7:string, pattern a), SelectColumnIsNotNull(col 9:timestamp)))
               predicate: (((cstring1 like '%b%') and cboolean2 is not null) or ((UDFToDouble(ctinyint) = cdouble) and (cstring2 like 'a') and ctimestamp2 is not null)) (type: boolean)
-              Statistics: Num rows: 7658 Data size: 1529972 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6879 Data size: 1304690 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int)
                 outputColumnNames: ctinyint, csmallint, cint
@@ -75,7 +75,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2]
-                Statistics: Num rows: 7658 Data size: 1529972 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6879 Data size: 1304690 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: max(csmallint), count(), min(csmallint), sum(cint), max(ctinyint)
                   Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/vectorization_7.q.out b/ql/src/test/results/clientpositive/vectorization_7.q.out
index 880a4c3..c27d13f 100644
--- a/ql/src/test/results/clientpositive/vectorization_7.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_7.q.out
@@ -71,7 +71,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28815.0D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 3019778 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2770698 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -81,7 +81,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterDoubleColGreaterDoubleScalar(col 5:double, val 988888.0), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28815.0)(children: CastTimestampToDouble(col 9:timestamp) -> 13:double), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 3569.0))), FilterExprOrExpr(children: FilterDoubleColLessEqualDoubleScalar(col 14:double, val -28800.0)(children: CastTimes [...]
               predicate: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28815.0D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-              Statistics: Num rows: 11033 Data size: 2711364 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 11033 Data size: 2487724 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cboolean1 (type: boolean), cbigint (type: bigint), csmallint (type: smallint), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cstring1 (type: string), (cbigint + cbigint) (type: bigint), (UDFToInteger(csmallint) % -257) (type: int), (- csmallint) (type: smallint), (- ctinyint) (type: tinyint), (UDFToInteger((- ctinyint)) + 17) (type: int), (cbigint * UDFToLong((- csmallint))) (type: bigint), (cint % UDFToInteger(csmallint)) (type: int), (- ctinyint) [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -90,7 +90,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [10, 3, 1, 0, 8, 6, 15, 16, 17, 18, 20, 22, 23, 24, 26]
                     selectExpressions: LongColAddLongColumn(col 3:bigint, col 3:bigint) -> 15:bigint, LongColModuloLongScalar(col 1:int, val -257)(children: col 1:smallint) -> 16:int, LongColUnaryMinus(col 1:smallint) -> 17:smallint, LongColUnaryMinus(col 0:tinyint) -> 18:tinyint, LongColAddLongScalar(col 19:int, val 17)(children: LongColUnaryMinus(col 0:tinyint) -> 19:tinyint) -> 20:int, LongColMultiplyLongColumn(col 3:bigint, col 21:bigint)(children: LongColUnaryMinus(col 1:smallint) - [...]
-                Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: boolean), _col1 (type: bigint), _col2 (type: smallint), _col3 (type: tinyint), _col4 (type: timestamp), _col5 (type: string), _col6 (type: bigint), _col7 (type: int), _col8 (type: smallint), _col9 (type: tinyint), _col10 (type: int), _col11 (type: bigint), _col12 (type: int), _col13 (type: tinyint), _col14 (type: tinyint)
                   sort order: +++++++++++++++
@@ -99,7 +99,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: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -125,13 +125,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: boolean), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: smallint), KEY.reducesinkkey3 (type: tinyint), KEY.reducesinkkey4 (type: timestamp), KEY.reducesinkkey5 (type: string), KEY.reducesinkkey6 (type: bigint), KEY.reducesinkkey7 (type: int), KEY.reducesinkkey8 (type: smallint), KEY.reducesinkkey9 (type: tinyint), KEY.reducesinkkey10 (type: int), KEY.reducesinkkey11 (type: bigint), KEY.reducesinkkey12 (type: int), KEY.reduce [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-          Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 25
-            Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 25 Data size: 4180 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 25 Data size: 4180 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -297,7 +297,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28792.315D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 3019778 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2770698 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -306,7 +306,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterDoubleColGreaterDoubleScalar(col 5:double, val 988888.0), FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 13:double, val -28792.315)(children: CastTimestampToDouble(col 9:timestamp) -> 13:double), FilterDoubleColLessEqualDoubleScalar(col 5:double, val 3569.0))), FilterExprOrExpr(children: FilterDoubleColLessEqualDoubleScalar(col 14:double, val -28800.0)(children: CastTim [...]
               predicate: (((cdouble > 988888.0D) or ((UDFToDouble(ctimestamp2) > -28792.315D) and (cdouble <= 3569.0D))) and ((UDFToDouble(ctimestamp1) <= -28800.0D) or (UDFToInteger(ctinyint) = cint) or (cstring2 like 'ss')) and (ctinyint <> 0Y)) (type: boolean)
-              Statistics: Num rows: 11033 Data size: 2711364 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 11033 Data size: 2487724 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cboolean1 (type: boolean), cbigint (type: bigint), csmallint (type: smallint), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cstring1 (type: string), (cbigint + cbigint) (type: bigint), (UDFToInteger(csmallint) % -257) (type: int), (- csmallint) (type: smallint), (- ctinyint) (type: tinyint), (UDFToInteger((- ctinyint)) + 17) (type: int), (cbigint * UDFToLong((- csmallint))) (type: bigint), (cint % UDFToInteger(csmallint)) (type: int), (- ctinyint) [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -315,7 +315,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [10, 3, 1, 0, 8, 6, 15, 16, 17, 18, 20, 22, 23, 24, 26]
                     selectExpressions: LongColAddLongColumn(col 3:bigint, col 3:bigint) -> 15:bigint, LongColModuloLongScalar(col 1:int, val -257)(children: col 1:smallint) -> 16:int, LongColUnaryMinus(col 1:smallint) -> 17:smallint, LongColUnaryMinus(col 0:tinyint) -> 18:tinyint, LongColAddLongScalar(col 19:int, val 17)(children: LongColUnaryMinus(col 0:tinyint) -> 19:tinyint) -> 20:int, LongColMultiplyLongColumn(col 3:bigint, col 21:bigint)(children: LongColUnaryMinus(col 1:smallint) - [...]
-                Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: boolean), _col1 (type: bigint), _col2 (type: smallint), _col3 (type: tinyint), _col4 (type: timestamp), _col5 (type: string), _col6 (type: bigint), _col7 (type: int), _col8 (type: smallint), _col9 (type: tinyint), _col10 (type: int), _col11 (type: bigint), _col12 (type: int), _col13 (type: tinyint), _col14 (type: tinyint)
                   sort order: +++++++++++++++
@@ -324,7 +324,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: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -344,13 +344,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: boolean), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: smallint), KEY.reducesinkkey3 (type: tinyint), KEY.reducesinkkey4 (type: timestamp), KEY.reducesinkkey5 (type: string), KEY.reducesinkkey6 (type: bigint), KEY.reducesinkkey7 (type: int), KEY.reducesinkkey8 (type: smallint), KEY.reducesinkkey9 (type: tinyint), KEY.reducesinkkey10 (type: int), KEY.reducesinkkey11 (type: bigint), KEY.reducesinkkey12 (type: int), KEY.reduce [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
-          Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 11033 Data size: 1754052 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 25
-            Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 25 Data size: 4180 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 25 Data size: 4180 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_8.q.out b/ql/src/test/results/clientpositive/vectorization_8.q.out
index 847064b..d5f635f 100644
--- a/ql/src/test/results/clientpositive/vectorization_8.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_8.q.out
@@ -67,7 +67,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 10.0D) and (UDFToDouble(ctimestamp2) <> 16.0D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2983078 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2733998 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -77,7 +77,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val -6432.0), FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleScalar(col 13:double, val 10.0)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val 16.0)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), SelectColumnIsNotNull(col 7:string)), FilterExprAndExpr(children: FilterDoubleColEqualDo [...]
               predicate: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 10.0D) and (UDFToDouble(ctimestamp2) <> 16.0D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-              Statistics: Num rows: 3059 Data size: 742850 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3059 Data size: 680930 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ctimestamp1 (type: timestamp), cdouble (type: double), cboolean1 (type: boolean), cstring1 (type: string), cfloat (type: float), (- cdouble) (type: double), (-5638.15D - cdouble) (type: double), (cdouble * -257.0D) (type: double), (UDFToFloat(cint) + cfloat) (type: float), ((- cdouble) + UDFToDouble(cbigint)) (type: double), (- cdouble) (type: double), (-1.389 - cfloat) (type: float), (- cfloat) (type: float), ((-5638.15D - cdouble) + UDFToDouble((UDFToFloat( [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -86,7 +86,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [8, 5, 10, 6, 4, 15, 16, 17, 19, 22, 23, 24, 25, 29]
                     selectExpressions: DoubleColUnaryMinus(col 5:double) -> 15:double, DoubleScalarSubtractDoubleColumn(val -5638.15, col 5:double) -> 16:double, DoubleColMultiplyDoubleScalar(col 5:double, val -257.0) -> 17:double, DoubleColAddDoubleColumn(col 18:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 18:float) -> 19:float, DoubleColAddDoubleColumn(col 20:double, col 21:double)(children: DoubleColUnaryMinus(col 5:double) -> 20:double, CastLongToDouble( [...]
-                Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: timestamp), _col1 (type: double), _col2 (type: boolean), _col3 (type: string), _col4 (type: float), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: float), _col13 (type: double)
                   sort order: ++++++++++++++
@@ -95,7 +95,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: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -121,13 +121,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: timestamp), KEY.reducesinkkey1 (type: double), KEY.reducesinkkey2 (type: boolean), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: float), KEY.reducesinkkey5 (type: double), KEY.reducesinkkey6 (type: double), KEY.reducesinkkey7 (type: double), KEY.reducesinkkey8 (type: float), KEY.reducesinkkey9 (type: double), KEY.reducesinkkey5 (type: double), KEY.reducesinkkey11 (type: float), KEY.reducesinkkey12 (type: float), KEY.reducesi [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
-          Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 20
-            Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3600 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 3600 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -280,7 +280,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 12.503D) and (UDFToDouble(ctimestamp2) <> 11.998D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2983078 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2733998 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -289,7 +289,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprOrExpr(children: FilterDoubleColLessDoubleScalar(col 4:float, val -6432.0), FilterExprAndExpr(children: FilterDoubleColLessEqualDoubleScalar(col 13:double, val 12.503)(children: CastTimestampToDouble(col 8:timestamp) -> 13:double), FilterDoubleColNotEqualDoubleScalar(col 14:double, val 11.998)(children: CastTimestampToDouble(col 9:timestamp) -> 14:double), SelectColumnIsNotNull(col 7:string)), FilterExprAndExpr(children: FilterDoubleColEqu [...]
               predicate: ((cfloat < -6432.0) or ((UDFToDouble(ctimestamp1) <= 12.503D) and (UDFToDouble(ctimestamp2) <> 11.998D) and cstring2 is not null) or ((cdouble = 988888.0D) and cboolean1 is not null)) (type: boolean)
-              Statistics: Num rows: 3059 Data size: 742850 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3059 Data size: 680930 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ctimestamp1 (type: timestamp), cdouble (type: double), cboolean1 (type: boolean), cstring1 (type: string), cfloat (type: float), (- cdouble) (type: double), (-5638.15D - cdouble) (type: double), (cdouble * -257.0D) (type: double), (UDFToFloat(cint) + cfloat) (type: float), ((- cdouble) + UDFToDouble(cbigint)) (type: double), (- cdouble) (type: double), (-1.389 - cfloat) (type: float), (- cfloat) (type: float), ((-5638.15D - cdouble) + UDFToDouble((UDFToFloat( [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -298,7 +298,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [8, 5, 10, 6, 4, 15, 16, 17, 19, 22, 23, 24, 25, 29]
                     selectExpressions: DoubleColUnaryMinus(col 5:double) -> 15:double, DoubleScalarSubtractDoubleColumn(val -5638.15, col 5:double) -> 16:double, DoubleColMultiplyDoubleScalar(col 5:double, val -257.0) -> 17:double, DoubleColAddDoubleColumn(col 18:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 2:int) -> 18:float) -> 19:float, DoubleColAddDoubleColumn(col 20:double, col 21:double)(children: DoubleColUnaryMinus(col 5:double) -> 20:double, CastLongToDouble( [...]
-                Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: timestamp), _col1 (type: double), _col2 (type: boolean), _col3 (type: string), _col4 (type: float), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: float), _col13 (type: double)
                   sort order: ++++++++++++++
@@ -307,7 +307,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: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -327,13 +327,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: timestamp), KEY.reducesinkkey1 (type: double), KEY.reducesinkkey2 (type: boolean), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: float), KEY.reducesinkkey5 (type: double), KEY.reducesinkkey6 (type: double), KEY.reducesinkkey7 (type: double), KEY.reducesinkkey8 (type: float), KEY.reducesinkkey9 (type: double), KEY.reducesinkkey5 (type: double), KEY.reducesinkkey11 (type: float), KEY.reducesinkkey12 (type: float), KEY.reducesi [...]
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
-          Statistics: Num rows: 3059 Data size: 557250 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 3059 Data size: 526290 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 20
-            Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3600 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 20 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 3600 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_9.q.out b/ql/src/test/results/clientpositive/vectorization_9.q.out
index 2dfcc77..9c782f2 100644
--- a/ql/src/test/results/clientpositive/vectorization_9.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_9.q.out
@@ -57,7 +57,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 2308074 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 2183514 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -67,7 +67,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterStringColLikeStringScalar(col 7:string, pattern %b%), FilterExprOrExpr(children: FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -1.389), FilterStringGroupColLessStringScalar(col 6:string, val a)))
               predicate: ((cstring2 like '%b%') and ((cdouble >= -1.389D) or (cstring1 < 'a'))) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cstring1 (type: string), cdouble (type: double), ctimestamp1 (type: timestamp), (cdouble * cdouble) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3
@@ -76,7 +76,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [6, 5, 8, 13]
                     selectExpressions: DoubleColMultiplyDoubleColumn(col 5:double, col 5:double) -> 13:double
-                Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 1091808 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(_col1), sum(_col3), sum(_col1), min(_col1)
                   Group By Vectorization:
@@ -91,7 +91,7 @@ STAGE PLANS:
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                  Statistics: Num rows: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp)
                     sort order: +++
@@ -101,7 +101,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: 3072 Data size: 455172 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col3 (type: bigint), _col4 (type: double), _col5 (type: double), _col6 (type: double)
       Execution mode: vectorized
       Map Vectorization:
@@ -129,14 +129,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: double), KEY._col2 (type: timestamp)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-          Statistics: Num rows: 1536 Data size: 227586 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 3072 Data size: 424052 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: string), _col1 (type: double), _col2 (type: timestamp), (_col1 - 9763215.5639D) (type: double), (- (_col1 - 9763215.5639D)) (type: double), _col3 (type: bigint), power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5) (type: double), (- power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5)) (type: double), (power(((_col4 - ((_col5 * _col5 [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-            Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 3072 Data size: 890996 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3072 Data size: 890996 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_casts.q.out b/ql/src/test/results/clientpositive/vectorized_casts.q.out
index 4e991dd..13bbb93 100644
--- a/ql/src/test/results/clientpositive/vectorized_casts.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_casts.q.out
@@ -167,7 +167,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cbigint % 250L) = 0L) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1684250 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1559690 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
                 vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -177,7 +177,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterLongColEqualLongScalar(col 13:bigint, val 0)(children: LongColModuloLongScalar(col 3:bigint, val 250) -> 13:bigint)
               predicate: ((cbigint % 250L) = 0L) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 842180 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 779900 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: UDFToBoolean(ctinyint) (type: boolean), UDFToBoolean(csmallint) (type: boolean), UDFToBoolean(cint) (type: boolean), UDFToBoolean(cbigint) (type: boolean), UDFToBoolean(cfloat) (type: boolean), UDFToBoolean(cdouble) (type: boolean), cboolean1 (type: boolean), ((cbigint * 0L) <> 0L) (type: boolean), UDFToBoolean(ctimestamp1) (type: boolean), UDFToBoolean(cstring1) (type: boolean), UDFToInteger(ctinyint) (type: int), UDFToInteger(csmallint) (type: int), cint (t [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50, _col51, _col52, _col53, _col54, _col55, _col56, _col57, _co [...]
@@ -186,13 +186,13 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [14, 15, 16, 17, 18, 19, 10, 21, 22, 23, 0, 1, 2, 3, 24, 25, 10, 26, 27, 29, 30, 31, 32, 33, 34, 35, 36, 4, 5, 37, 38, 39, 41, 42, 5, 44, 46, 48, 50, 51, 52, 54, 58, 60, 8, 61, 63, 64, 65, 66, 67, 68, 69, 70, 72, 73, 6, 74, 75, 77, 79, 81, 84]
                     selectExpressions: CastLongToBooleanViaLongToLong(col 0:tinyint) -> 14:boolean, CastLongToBooleanViaLongToLong(col 1:smallint) -> 15:boolean, CastLongToBooleanViaLongToLong(col 2:int) -> 16:boolean, CastLongToBooleanViaLongToLong(col 3:bigint) -> 17:boolean, CastDoubleToBooleanViaDoubleToLong(col 4:float) -> 18:boolean, CastDoubleToBooleanViaDoubleToLong(col 5:double) -> 19:boolean, LongColNotEqualLongScalar(col 20:bigint, val 0)(children: LongColMultiplyLongScalar(co [...]
-                Statistics: Num rows: 6144 Data size: 16362860 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 16014092 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 6144 Data size: 16362860 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6144 Data size: 16014092 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 84c4255..5c1fe26 100644
--- a/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
@@ -257,7 +257,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 52 Data size: 4276 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 52 Data size: 3956 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Select Operator
@@ -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: 16756 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
                 key expressions: _col0 (type: bigint)
                 sort order: +
@@ -277,7 +277,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: 16756 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 52 Data size: 16436 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:
@@ -297,10 +297,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: 16756 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 52 Data size: 16756 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 52 Data size: 16436 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -635,7 +635,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 52 Data size: 7617 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 52 Data size: 7497 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Select Operator
@@ -936,7 +936,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Select Operator
@@ -946,7 +946,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [1]
-              Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: min(ctimestamp1), max(ctimestamp1), count(ctimestamp1), count()
                 Group By Vectorization:
@@ -1048,7 +1048,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Select Operator
@@ -1058,7 +1058,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [1]
-              Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: sum(ctimestamp1)
                 Group By Vectorization:
@@ -1172,7 +1172,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypesorc_string
-            Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Select Operator
@@ -1183,7 +1183,7 @@ STAGE PLANS:
                   native: true
                   projectedOutputColumnNums: [1, 5, 8]
                   selectExpressions: CastTimestampToDouble(col 1:timestamp) -> 5:double, DoubleColMultiplyDoubleColumn(col 6:double, col 7:double)(children: CastTimestampToDouble(col 1:timestamp) -> 6:double, CastTimestampToDouble(col 1:timestamp) -> 7:double) -> 8:double
-              Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 52 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: sum(_col0), count(_col0), sum(_col2), sum(_col1)
                 Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/vectorized_timestamp_ints_casts.q.out b/ql/src/test/results/clientpositive/vectorized_timestamp_ints_casts.q.out
index e9dbd88..f6057ab 100644
--- a/ql/src/test/results/clientpositive/vectorized_timestamp_ints_casts.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_timestamp_ints_casts.q.out
@@ -53,7 +53,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cbigint % 250L) = 0L) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1684250 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1559690 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -62,7 +62,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterLongColEqualLongScalar(col 13:bigint, val 0)(children: LongColModuloLongScalar(col 3:bigint, val 250) -> 13:bigint)
               predicate: ((cbigint % 250L) = 0L) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 842180 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 779900 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0L) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp),  [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -71,13 +71,13 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [15, 17, 19, 21, 22, 23, 25, 29, 8, 30, 32]
                     selectExpressions: CastMillisecondsLongToTimestamp(col 0:tinyint) -> 15:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 17:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 19:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 21:timestamp, CastDoubleToTimestamp(col 4:float) -> 22:timestamp, CastDoubleToTimestamp(col 5:double) -> 23:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 25:timestamp, CastMillisecondsLongToTimestamp [...]
-                Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 2641080 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6144 Data size: 2641080 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -218,7 +218,7 @@ STAGE PLANS:
           TableScan
             alias: alltypesorc
             filterExpr: ((cbigint % 250L) = 0L) (type: boolean)
-            Statistics: Num rows: 12288 Data size: 1684250 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12288 Data size: 1559690 Basic stats: COMPLETE Column stats: COMPLETE
             TableScan Vectorization:
                 native: true
             Filter Operator
@@ -227,7 +227,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterLongColEqualLongScalar(col 13:bigint, val 0)(children: LongColModuloLongScalar(col 3:bigint, val 250) -> 13:bigint)
               predicate: ((cbigint % 250L) = 0L) (type: boolean)
-              Statistics: Num rows: 6144 Data size: 842180 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 779900 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0L) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp),  [...]
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -236,13 +236,13 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [14, 15, 16, 17, 18, 19, 20, 22, 8, 23, 25]
                     selectExpressions: CastLongToTimestamp(col 0:tinyint) -> 14:timestamp, CastLongToTimestamp(col 1:smallint) -> 15:timestamp, CastLongToTimestamp(col 2:int) -> 16:timestamp, CastLongToTimestamp(col 3:bigint) -> 17:timestamp, CastDoubleToTimestamp(col 4:float) -> 18:timestamp, CastDoubleToTimestamp(col 5:double) -> 19:timestamp, CastLongToTimestamp(col 10:boolean) -> 20:timestamp, CastLongToTimestamp(col 21:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) [...]
-                Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 2641080 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6144 Data size: 2641080 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-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsData.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsData.java
index 9a2e4f4..a92a867 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsData.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsData.java
@@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField BINARY_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryStats", org.apache.thrift.protocol.TType.STRUCT, (short)5);
   private static final org.apache.thrift.protocol.TField DECIMAL_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("decimalStats", org.apache.thrift.protocol.TType.STRUCT, (short)6);
   private static final org.apache.thrift.protocol.TField DATE_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("dateStats", org.apache.thrift.protocol.TType.STRUCT, (short)7);
+  private static final org.apache.thrift.protocol.TField TIMESTAMP_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("timestampStats", org.apache.thrift.protocol.TType.STRUCT, (short)8);
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -52,7 +53,8 @@ import org.slf4j.LoggerFactory;
     STRING_STATS((short)4, "stringStats"),
     BINARY_STATS((short)5, "binaryStats"),
     DECIMAL_STATS((short)6, "decimalStats"),
-    DATE_STATS((short)7, "dateStats");
+    DATE_STATS((short)7, "dateStats"),
+    TIMESTAMP_STATS((short)8, "timestampStats");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -81,6 +83,8 @@ import org.slf4j.LoggerFactory;
           return DECIMAL_STATS;
         case 7: // DATE_STATS
           return DATE_STATS;
+        case 8: // TIMESTAMP_STATS
+          return TIMESTAMP_STATS;
         default:
           return null;
       }
@@ -137,6 +141,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DecimalColumnStatsData.class)));
     tmpMap.put(_Fields.DATE_STATS, new org.apache.thrift.meta_data.FieldMetaData("dateStats", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DateColumnStatsData.class)));
+    tmpMap.put(_Fields.TIMESTAMP_STATS, new org.apache.thrift.meta_data.FieldMetaData("timestampStats", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TimestampColumnStatsData.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ColumnStatisticsData.class, metaDataMap);
   }
@@ -198,6 +204,12 @@ import org.slf4j.LoggerFactory;
     return x;
   }
 
+  public static ColumnStatisticsData timestampStats(TimestampColumnStatsData value) {
+    ColumnStatisticsData x = new ColumnStatisticsData();
+    x.setTimestampStats(value);
+    return x;
+  }
+
 
   @Override
   protected void checkType(_Fields setField, Object value) throws ClassCastException {
@@ -237,6 +249,11 @@ import org.slf4j.LoggerFactory;
           break;
         }
         throw new ClassCastException("Was expecting value of type DateColumnStatsData for field 'dateStats', but got " + value.getClass().getSimpleName());
+      case TIMESTAMP_STATS:
+        if (value instanceof TimestampColumnStatsData) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type TimestampColumnStatsData for field 'timestampStats', but got " + value.getClass().getSimpleName());
       default:
         throw new IllegalArgumentException("Unknown field id " + setField);
     }
@@ -317,6 +334,16 @@ import org.slf4j.LoggerFactory;
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             return null;
           }
+        case TIMESTAMP_STATS:
+          if (field.type == TIMESTAMP_STATS_FIELD_DESC.type) {
+            TimestampColumnStatsData timestampStats;
+            timestampStats = new TimestampColumnStatsData();
+            timestampStats.read(iprot);
+            return timestampStats;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
         default:
           throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
       }
@@ -357,6 +384,10 @@ import org.slf4j.LoggerFactory;
         DateColumnStatsData dateStats = (DateColumnStatsData)value_;
         dateStats.write(oprot);
         return;
+      case TIMESTAMP_STATS:
+        TimestampColumnStatsData timestampStats = (TimestampColumnStatsData)value_;
+        timestampStats.write(oprot);
+        return;
       default:
         throw new IllegalStateException("Cannot write union with unknown field " + setField_);
     }
@@ -402,6 +433,11 @@ import org.slf4j.LoggerFactory;
           dateStats = new DateColumnStatsData();
           dateStats.read(iprot);
           return dateStats;
+        case TIMESTAMP_STATS:
+          TimestampColumnStatsData timestampStats;
+          timestampStats = new TimestampColumnStatsData();
+          timestampStats.read(iprot);
+          return timestampStats;
         default:
           throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
       }
@@ -441,6 +477,10 @@ import org.slf4j.LoggerFactory;
         DateColumnStatsData dateStats = (DateColumnStatsData)value_;
         dateStats.write(oprot);
         return;
+      case TIMESTAMP_STATS:
+        TimestampColumnStatsData timestampStats = (TimestampColumnStatsData)value_;
+        timestampStats.write(oprot);
+        return;
       default:
         throw new IllegalStateException("Cannot write union with unknown field " + setField_);
     }
@@ -463,6 +503,8 @@ import org.slf4j.LoggerFactory;
         return DECIMAL_STATS_FIELD_DESC;
       case DATE_STATS:
         return DATE_STATS_FIELD_DESC;
+      case TIMESTAMP_STATS:
+        return TIMESTAMP_STATS_FIELD_DESC;
       default:
         throw new IllegalArgumentException("Unknown field id " + setField);
     }
@@ -581,6 +623,20 @@ import org.slf4j.LoggerFactory;
     value_ = value;
   }
 
+  public TimestampColumnStatsData getTimestampStats() {
+    if (getSetField() == _Fields.TIMESTAMP_STATS) {
+      return (TimestampColumnStatsData)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'timestampStats' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void setTimestampStats(TimestampColumnStatsData value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.TIMESTAMP_STATS;
+    value_ = value;
+  }
+
   public boolean isSetBooleanStats() {
     return setField_ == _Fields.BOOLEAN_STATS;
   }
@@ -616,6 +672,11 @@ import org.slf4j.LoggerFactory;
   }
 
 
+  public boolean isSetTimestampStats() {
+    return setField_ == _Fields.TIMESTAMP_STATS;
+  }
+
+
   public boolean equals(Object other) {
     if (other instanceof ColumnStatisticsData) {
       return equals((ColumnStatisticsData)other);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Timestamp.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Timestamp.java
new file mode 100644
index 0000000..b43eb1c
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Timestamp.java
@@ -0,0 +1,387 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class Timestamp implements org.apache.thrift.TBase<Timestamp, Timestamp._Fields>, java.io.Serializable, Cloneable, Comparable<Timestamp> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Timestamp");
+
+  private static final org.apache.thrift.protocol.TField SECONDS_SINCE_EPOCH_FIELD_DESC = new org.apache.thrift.protocol.TField("secondsSinceEpoch", org.apache.thrift.protocol.TType.I64, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TimestampStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TimestampTupleSchemeFactory());
+  }
+
+  private long secondsSinceEpoch; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SECONDS_SINCE_EPOCH((short)1, "secondsSinceEpoch");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SECONDS_SINCE_EPOCH
+          return SECONDS_SINCE_EPOCH;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __SECONDSSINCEEPOCH_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SECONDS_SINCE_EPOCH, new org.apache.thrift.meta_data.FieldMetaData("secondsSinceEpoch", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Timestamp.class, metaDataMap);
+  }
+
+  public Timestamp() {
+  }
+
+  public Timestamp(
+    long secondsSinceEpoch)
+  {
+    this();
+    this.secondsSinceEpoch = secondsSinceEpoch;
+    setSecondsSinceEpochIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public Timestamp(Timestamp other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.secondsSinceEpoch = other.secondsSinceEpoch;
+  }
+
+  public Timestamp deepCopy() {
+    return new Timestamp(this);
+  }
+
+  @Override
+  public void clear() {
+    setSecondsSinceEpochIsSet(false);
+    this.secondsSinceEpoch = 0;
+  }
+
+  public long getSecondsSinceEpoch() {
+    return this.secondsSinceEpoch;
+  }
+
+  public void setSecondsSinceEpoch(long secondsSinceEpoch) {
+    this.secondsSinceEpoch = secondsSinceEpoch;
+    setSecondsSinceEpochIsSet(true);
+  }
+
+  public void unsetSecondsSinceEpoch() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SECONDSSINCEEPOCH_ISSET_ID);
+  }
+
+  /** Returns true if field secondsSinceEpoch is set (has been assigned a value) and false otherwise */
+  public boolean isSetSecondsSinceEpoch() {
+    return EncodingUtils.testBit(__isset_bitfield, __SECONDSSINCEEPOCH_ISSET_ID);
+  }
+
+  public void setSecondsSinceEpochIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SECONDSSINCEEPOCH_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SECONDS_SINCE_EPOCH:
+      if (value == null) {
+        unsetSecondsSinceEpoch();
+      } else {
+        setSecondsSinceEpoch((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SECONDS_SINCE_EPOCH:
+      return getSecondsSinceEpoch();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SECONDS_SINCE_EPOCH:
+      return isSetSecondsSinceEpoch();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof Timestamp)
+      return this.equals((Timestamp)that);
+    return false;
+  }
+
+  public boolean equals(Timestamp that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_secondsSinceEpoch = true;
+    boolean that_present_secondsSinceEpoch = true;
+    if (this_present_secondsSinceEpoch || that_present_secondsSinceEpoch) {
+      if (!(this_present_secondsSinceEpoch && that_present_secondsSinceEpoch))
+        return false;
+      if (this.secondsSinceEpoch != that.secondsSinceEpoch)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_secondsSinceEpoch = true;
+    list.add(present_secondsSinceEpoch);
+    if (present_secondsSinceEpoch)
+      list.add(secondsSinceEpoch);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(Timestamp other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetSecondsSinceEpoch()).compareTo(other.isSetSecondsSinceEpoch());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSecondsSinceEpoch()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.secondsSinceEpoch, other.secondsSinceEpoch);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Timestamp(");
+    boolean first = true;
+
+    sb.append("secondsSinceEpoch:");
+    sb.append(this.secondsSinceEpoch);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetSecondsSinceEpoch()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'secondsSinceEpoch' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TimestampStandardSchemeFactory implements SchemeFactory {
+    public TimestampStandardScheme getScheme() {
+      return new TimestampStandardScheme();
+    }
+  }
+
+  private static class TimestampStandardScheme extends StandardScheme<Timestamp> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, Timestamp struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SECONDS_SINCE_EPOCH
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.secondsSinceEpoch = iprot.readI64();
+              struct.setSecondsSinceEpochIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, Timestamp struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(SECONDS_SINCE_EPOCH_FIELD_DESC);
+      oprot.writeI64(struct.secondsSinceEpoch);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TimestampTupleSchemeFactory implements SchemeFactory {
+    public TimestampTupleScheme getScheme() {
+      return new TimestampTupleScheme();
+    }
+  }
+
+  private static class TimestampTupleScheme extends TupleScheme<Timestamp> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, Timestamp struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.secondsSinceEpoch);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, Timestamp struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.secondsSinceEpoch = iprot.readI64();
+      struct.setSecondsSinceEpochIsSet(true);
+    }
+  }
+
+}
+
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TimestampColumnStatsData.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TimestampColumnStatsData.java
new file mode 100644
index 0000000..3e250f5
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TimestampColumnStatsData.java
@@ -0,0 +1,823 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class TimestampColumnStatsData implements org.apache.thrift.TBase<TimestampColumnStatsData, TimestampColumnStatsData._Fields>, java.io.Serializable, Cloneable, Comparable<TimestampColumnStatsData> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TimestampColumnStatsData");
+
+  private static final org.apache.thrift.protocol.TField LOW_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("lowValue", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField HIGH_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("highValue", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField NUM_NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("numNulls", org.apache.thrift.protocol.TType.I64, (short)3);
+  private static final org.apache.thrift.protocol.TField NUM_DVS_FIELD_DESC = new org.apache.thrift.protocol.TField("numDVs", org.apache.thrift.protocol.TType.I64, (short)4);
+  private static final org.apache.thrift.protocol.TField BIT_VECTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("bitVectors", org.apache.thrift.protocol.TType.STRING, (short)5);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TimestampColumnStatsDataStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TimestampColumnStatsDataTupleSchemeFactory());
+  }
+
+  private Timestamp lowValue; // optional
+  private Timestamp highValue; // optional
+  private long numNulls; // required
+  private long numDVs; // required
+  private ByteBuffer bitVectors; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    LOW_VALUE((short)1, "lowValue"),
+    HIGH_VALUE((short)2, "highValue"),
+    NUM_NULLS((short)3, "numNulls"),
+    NUM_DVS((short)4, "numDVs"),
+    BIT_VECTORS((short)5, "bitVectors");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // LOW_VALUE
+          return LOW_VALUE;
+        case 2: // HIGH_VALUE
+          return HIGH_VALUE;
+        case 3: // NUM_NULLS
+          return NUM_NULLS;
+        case 4: // NUM_DVS
+          return NUM_DVS;
+        case 5: // BIT_VECTORS
+          return BIT_VECTORS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NUMNULLS_ISSET_ID = 0;
+  private static final int __NUMDVS_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.LOW_VALUE,_Fields.HIGH_VALUE,_Fields.BIT_VECTORS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.LOW_VALUE, new org.apache.thrift.meta_data.FieldMetaData("lowValue", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Timestamp.class)));
+    tmpMap.put(_Fields.HIGH_VALUE, new org.apache.thrift.meta_data.FieldMetaData("highValue", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Timestamp.class)));
+    tmpMap.put(_Fields.NUM_NULLS, new org.apache.thrift.meta_data.FieldMetaData("numNulls", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.NUM_DVS, new org.apache.thrift.meta_data.FieldMetaData("numDVs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.BIT_VECTORS, new org.apache.thrift.meta_data.FieldMetaData("bitVectors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TimestampColumnStatsData.class, metaDataMap);
+  }
+
+  public TimestampColumnStatsData() {
+  }
+
+  public TimestampColumnStatsData(
+    long numNulls,
+    long numDVs)
+  {
+    this();
+    this.numNulls = numNulls;
+    setNumNullsIsSet(true);
+    this.numDVs = numDVs;
+    setNumDVsIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TimestampColumnStatsData(TimestampColumnStatsData other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetLowValue()) {
+      this.lowValue = new Timestamp(other.lowValue);
+    }
+    if (other.isSetHighValue()) {
+      this.highValue = new Timestamp(other.highValue);
+    }
+    this.numNulls = other.numNulls;
+    this.numDVs = other.numDVs;
+    if (other.isSetBitVectors()) {
+      this.bitVectors = org.apache.thrift.TBaseHelper.copyBinary(other.bitVectors);
+    }
+  }
+
+  public TimestampColumnStatsData deepCopy() {
+    return new TimestampColumnStatsData(this);
+  }
+
+  @Override
+  public void clear() {
+    this.lowValue = null;
+    this.highValue = null;
+    setNumNullsIsSet(false);
+    this.numNulls = 0;
+    setNumDVsIsSet(false);
+    this.numDVs = 0;
+    this.bitVectors = null;
+  }
+
+  public Timestamp getLowValue() {
+    return this.lowValue;
+  }
+
+  public void setLowValue(Timestamp lowValue) {
+    this.lowValue = lowValue;
+  }
+
+  public void unsetLowValue() {
+    this.lowValue = null;
+  }
+
+  /** Returns true if field lowValue is set (has been assigned a value) and false otherwise */
+  public boolean isSetLowValue() {
+    return this.lowValue != null;
+  }
+
+  public void setLowValueIsSet(boolean value) {
+    if (!value) {
+      this.lowValue = null;
+    }
+  }
+
+  public Timestamp getHighValue() {
+    return this.highValue;
+  }
+
+  public void setHighValue(Timestamp highValue) {
+    this.highValue = highValue;
+  }
+
+  public void unsetHighValue() {
+    this.highValue = null;
+  }
+
+  /** Returns true if field highValue is set (has been assigned a value) and false otherwise */
+  public boolean isSetHighValue() {
+    return this.highValue != null;
+  }
+
+  public void setHighValueIsSet(boolean value) {
+    if (!value) {
+      this.highValue = null;
+    }
+  }
+
+  public long getNumNulls() {
+    return this.numNulls;
+  }
+
+  public void setNumNulls(long numNulls) {
+    this.numNulls = numNulls;
+    setNumNullsIsSet(true);
+  }
+
+  public void unsetNumNulls() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUMNULLS_ISSET_ID);
+  }
+
+  /** Returns true if field numNulls is set (has been assigned a value) and false otherwise */
+  public boolean isSetNumNulls() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUMNULLS_ISSET_ID);
+  }
+
+  public void setNumNullsIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUMNULLS_ISSET_ID, value);
+  }
+
+  public long getNumDVs() {
+    return this.numDVs;
+  }
+
+  public void setNumDVs(long numDVs) {
+    this.numDVs = numDVs;
+    setNumDVsIsSet(true);
+  }
+
+  public void unsetNumDVs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUMDVS_ISSET_ID);
+  }
+
+  /** Returns true if field numDVs is set (has been assigned a value) and false otherwise */
+  public boolean isSetNumDVs() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUMDVS_ISSET_ID);
+  }
+
+  public void setNumDVsIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUMDVS_ISSET_ID, value);
+  }
+
+  public byte[] getBitVectors() {
+    setBitVectors(org.apache.thrift.TBaseHelper.rightSize(bitVectors));
+    return bitVectors == null ? null : bitVectors.array();
+  }
+
+  public ByteBuffer bufferForBitVectors() {
+    return org.apache.thrift.TBaseHelper.copyBinary(bitVectors);
+  }
+
+  public void setBitVectors(byte[] bitVectors) {
+    this.bitVectors = bitVectors == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(bitVectors, bitVectors.length));
+  }
+
+  public void setBitVectors(ByteBuffer bitVectors) {
+    this.bitVectors = org.apache.thrift.TBaseHelper.copyBinary(bitVectors);
+  }
+
+  public void unsetBitVectors() {
+    this.bitVectors = null;
+  }
+
+  /** Returns true if field bitVectors is set (has been assigned a value) and false otherwise */
+  public boolean isSetBitVectors() {
+    return this.bitVectors != null;
+  }
+
+  public void setBitVectorsIsSet(boolean value) {
+    if (!value) {
+      this.bitVectors = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case LOW_VALUE:
+      if (value == null) {
+        unsetLowValue();
+      } else {
+        setLowValue((Timestamp)value);
+      }
+      break;
+
+    case HIGH_VALUE:
+      if (value == null) {
+        unsetHighValue();
+      } else {
+        setHighValue((Timestamp)value);
+      }
+      break;
+
+    case NUM_NULLS:
+      if (value == null) {
+        unsetNumNulls();
+      } else {
+        setNumNulls((Long)value);
+      }
+      break;
+
+    case NUM_DVS:
+      if (value == null) {
+        unsetNumDVs();
+      } else {
+        setNumDVs((Long)value);
+      }
+      break;
+
+    case BIT_VECTORS:
+      if (value == null) {
+        unsetBitVectors();
+      } else {
+        setBitVectors((ByteBuffer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case LOW_VALUE:
+      return getLowValue();
+
+    case HIGH_VALUE:
+      return getHighValue();
+
+    case NUM_NULLS:
+      return getNumNulls();
+
+    case NUM_DVS:
+      return getNumDVs();
+
+    case BIT_VECTORS:
+      return getBitVectors();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case LOW_VALUE:
+      return isSetLowValue();
+    case HIGH_VALUE:
+      return isSetHighValue();
+    case NUM_NULLS:
+      return isSetNumNulls();
+    case NUM_DVS:
+      return isSetNumDVs();
+    case BIT_VECTORS:
+      return isSetBitVectors();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TimestampColumnStatsData)
+      return this.equals((TimestampColumnStatsData)that);
+    return false;
+  }
+
+  public boolean equals(TimestampColumnStatsData that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_lowValue = true && this.isSetLowValue();
+    boolean that_present_lowValue = true && that.isSetLowValue();
+    if (this_present_lowValue || that_present_lowValue) {
+      if (!(this_present_lowValue && that_present_lowValue))
+        return false;
+      if (!this.lowValue.equals(that.lowValue))
+        return false;
+    }
+
+    boolean this_present_highValue = true && this.isSetHighValue();
+    boolean that_present_highValue = true && that.isSetHighValue();
+    if (this_present_highValue || that_present_highValue) {
+      if (!(this_present_highValue && that_present_highValue))
+        return false;
+      if (!this.highValue.equals(that.highValue))
+        return false;
+    }
+
+    boolean this_present_numNulls = true;
+    boolean that_present_numNulls = true;
+    if (this_present_numNulls || that_present_numNulls) {
+      if (!(this_present_numNulls && that_present_numNulls))
+        return false;
+      if (this.numNulls != that.numNulls)
+        return false;
+    }
+
+    boolean this_present_numDVs = true;
+    boolean that_present_numDVs = true;
+    if (this_present_numDVs || that_present_numDVs) {
+      if (!(this_present_numDVs && that_present_numDVs))
+        return false;
+      if (this.numDVs != that.numDVs)
+        return false;
+    }
+
+    boolean this_present_bitVectors = true && this.isSetBitVectors();
+    boolean that_present_bitVectors = true && that.isSetBitVectors();
+    if (this_present_bitVectors || that_present_bitVectors) {
+      if (!(this_present_bitVectors && that_present_bitVectors))
+        return false;
+      if (!this.bitVectors.equals(that.bitVectors))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_lowValue = true && (isSetLowValue());
+    list.add(present_lowValue);
+    if (present_lowValue)
+      list.add(lowValue);
+
+    boolean present_highValue = true && (isSetHighValue());
+    list.add(present_highValue);
+    if (present_highValue)
+      list.add(highValue);
+
+    boolean present_numNulls = true;
+    list.add(present_numNulls);
+    if (present_numNulls)
+      list.add(numNulls);
+
+    boolean present_numDVs = true;
+    list.add(present_numDVs);
+    if (present_numDVs)
+      list.add(numDVs);
+
+    boolean present_bitVectors = true && (isSetBitVectors());
+    list.add(present_bitVectors);
+    if (present_bitVectors)
+      list.add(bitVectors);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TimestampColumnStatsData other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetLowValue()).compareTo(other.isSetLowValue());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetLowValue()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lowValue, other.lowValue);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetHighValue()).compareTo(other.isSetHighValue());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetHighValue()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.highValue, other.highValue);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetNumNulls()).compareTo(other.isSetNumNulls());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetNumNulls()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.numNulls, other.numNulls);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetNumDVs()).compareTo(other.isSetNumDVs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetNumDVs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.numDVs, other.numDVs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetBitVectors()).compareTo(other.isSetBitVectors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetBitVectors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bitVectors, other.bitVectors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TimestampColumnStatsData(");
+    boolean first = true;
+
+    if (isSetLowValue()) {
+      sb.append("lowValue:");
+      if (this.lowValue == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.lowValue);
+      }
+      first = false;
+    }
+    if (isSetHighValue()) {
+      if (!first) sb.append(", ");
+      sb.append("highValue:");
+      if (this.highValue == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.highValue);
+      }
+      first = false;
+    }
+    if (!first) sb.append(", ");
+    sb.append("numNulls:");
+    sb.append(this.numNulls);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("numDVs:");
+    sb.append(this.numDVs);
+    first = false;
+    if (isSetBitVectors()) {
+      if (!first) sb.append(", ");
+      sb.append("bitVectors:");
+      if (this.bitVectors == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.bitVectors, sb);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetNumNulls()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'numNulls' is unset! Struct:" + toString());
+    }
+
+    if (!isSetNumDVs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'numDVs' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (lowValue != null) {
+      lowValue.validate();
+    }
+    if (highValue != null) {
+      highValue.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TimestampColumnStatsDataStandardSchemeFactory implements SchemeFactory {
+    public TimestampColumnStatsDataStandardScheme getScheme() {
+      return new TimestampColumnStatsDataStandardScheme();
+    }
+  }
+
+  private static class TimestampColumnStatsDataStandardScheme extends StandardScheme<TimestampColumnStatsData> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TimestampColumnStatsData struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // LOW_VALUE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.lowValue = new Timestamp();
+              struct.lowValue.read(iprot);
+              struct.setLowValueIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // HIGH_VALUE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.highValue = new Timestamp();
+              struct.highValue.read(iprot);
+              struct.setHighValueIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // NUM_NULLS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.numNulls = iprot.readI64();
+              struct.setNumNullsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // NUM_DVS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.numDVs = iprot.readI64();
+              struct.setNumDVsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // BIT_VECTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.bitVectors = iprot.readBinary();
+              struct.setBitVectorsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TimestampColumnStatsData struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.lowValue != null) {
+        if (struct.isSetLowValue()) {
+          oprot.writeFieldBegin(LOW_VALUE_FIELD_DESC);
+          struct.lowValue.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.highValue != null) {
+        if (struct.isSetHighValue()) {
+          oprot.writeFieldBegin(HIGH_VALUE_FIELD_DESC);
+          struct.highValue.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldBegin(NUM_NULLS_FIELD_DESC);
+      oprot.writeI64(struct.numNulls);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(NUM_DVS_FIELD_DESC);
+      oprot.writeI64(struct.numDVs);
+      oprot.writeFieldEnd();
+      if (struct.bitVectors != null) {
+        if (struct.isSetBitVectors()) {
+          oprot.writeFieldBegin(BIT_VECTORS_FIELD_DESC);
+          oprot.writeBinary(struct.bitVectors);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TimestampColumnStatsDataTupleSchemeFactory implements SchemeFactory {
+    public TimestampColumnStatsDataTupleScheme getScheme() {
+      return new TimestampColumnStatsDataTupleScheme();
+    }
+  }
+
+  private static class TimestampColumnStatsDataTupleScheme extends TupleScheme<TimestampColumnStatsData> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TimestampColumnStatsData struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.numNulls);
+      oprot.writeI64(struct.numDVs);
+      BitSet optionals = new BitSet();
+      if (struct.isSetLowValue()) {
+        optionals.set(0);
+      }
+      if (struct.isSetHighValue()) {
+        optionals.set(1);
+      }
+      if (struct.isSetBitVectors()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetLowValue()) {
+        struct.lowValue.write(oprot);
+      }
+      if (struct.isSetHighValue()) {
+        struct.highValue.write(oprot);
+      }
+      if (struct.isSetBitVectors()) {
+        oprot.writeBinary(struct.bitVectors);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TimestampColumnStatsData struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.numNulls = iprot.readI64();
+      struct.setNumNullsIsSet(true);
+      struct.numDVs = iprot.readI64();
+      struct.setNumDVsIsSet(true);
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.lowValue = new Timestamp();
+        struct.lowValue.read(iprot);
+        struct.setLowValueIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.highValue = new Timestamp();
+        struct.highValue.read(iprot);
+        struct.setHighValueIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.bitVectors = iprot.readBinary();
+        struct.setBitVectorsIsSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
index 03aa9bd..c86912b 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
@@ -9856,6 +9856,258 @@ class DateColumnStatsData {
 
 }
 
+class Timestamp {
+  static $_TSPEC;
+
+  /**
+   * @var int
+   */
+  public $secondsSinceEpoch = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'secondsSinceEpoch',
+          'type' => TType::I64,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['secondsSinceEpoch'])) {
+        $this->secondsSinceEpoch = $vals['secondsSinceEpoch'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'Timestamp';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->secondsSinceEpoch);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('Timestamp');
+    if ($this->secondsSinceEpoch !== null) {
+      $xfer += $output->writeFieldBegin('secondsSinceEpoch', TType::I64, 1);
+      $xfer += $output->writeI64($this->secondsSinceEpoch);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class TimestampColumnStatsData {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\Timestamp
+   */
+  public $lowValue = null;
+  /**
+   * @var \metastore\Timestamp
+   */
+  public $highValue = null;
+  /**
+   * @var int
+   */
+  public $numNulls = null;
+  /**
+   * @var int
+   */
+  public $numDVs = null;
+  /**
+   * @var string
+   */
+  public $bitVectors = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'lowValue',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Timestamp',
+          ),
+        2 => array(
+          'var' => 'highValue',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Timestamp',
+          ),
+        3 => array(
+          'var' => 'numNulls',
+          'type' => TType::I64,
+          ),
+        4 => array(
+          'var' => 'numDVs',
+          'type' => TType::I64,
+          ),
+        5 => array(
+          'var' => 'bitVectors',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['lowValue'])) {
+        $this->lowValue = $vals['lowValue'];
+      }
+      if (isset($vals['highValue'])) {
+        $this->highValue = $vals['highValue'];
+      }
+      if (isset($vals['numNulls'])) {
+        $this->numNulls = $vals['numNulls'];
+      }
+      if (isset($vals['numDVs'])) {
+        $this->numDVs = $vals['numDVs'];
+      }
+      if (isset($vals['bitVectors'])) {
+        $this->bitVectors = $vals['bitVectors'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'TimestampColumnStatsData';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->lowValue = new \metastore\Timestamp();
+            $xfer += $this->lowValue->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->highValue = new \metastore\Timestamp();
+            $xfer += $this->highValue->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->numNulls);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->numDVs);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->bitVectors);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('TimestampColumnStatsData');
+    if ($this->lowValue !== null) {
+      if (!is_object($this->lowValue)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('lowValue', TType::STRUCT, 1);
+      $xfer += $this->lowValue->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->highValue !== null) {
+      if (!is_object($this->highValue)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('highValue', TType::STRUCT, 2);
+      $xfer += $this->highValue->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->numNulls !== null) {
+      $xfer += $output->writeFieldBegin('numNulls', TType::I64, 3);
+      $xfer += $output->writeI64($this->numNulls);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->numDVs !== null) {
+      $xfer += $output->writeFieldBegin('numDVs', TType::I64, 4);
+      $xfer += $output->writeI64($this->numDVs);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->bitVectors !== null) {
+      $xfer += $output->writeFieldBegin('bitVectors', TType::STRING, 5);
+      $xfer += $output->writeString($this->bitVectors);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ColumnStatisticsData {
   static $_TSPEC;
 
@@ -9887,6 +10139,10 @@ class ColumnStatisticsData {
    * @var \metastore\DateColumnStatsData
    */
   public $dateStats = null;
+  /**
+   * @var \metastore\TimestampColumnStatsData
+   */
+  public $timestampStats = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -9926,6 +10182,11 @@ class ColumnStatisticsData {
           'type' => TType::STRUCT,
           'class' => '\metastore\DateColumnStatsData',
           ),
+        8 => array(
+          'var' => 'timestampStats',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\TimestampColumnStatsData',
+          ),
         );
     }
     if (is_array($vals)) {
@@ -9950,6 +10211,9 @@ class ColumnStatisticsData {
       if (isset($vals['dateStats'])) {
         $this->dateStats = $vals['dateStats'];
       }
+      if (isset($vals['timestampStats'])) {
+        $this->timestampStats = $vals['timestampStats'];
+      }
     }
   }
 
@@ -10028,6 +10292,14 @@ class ColumnStatisticsData {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 8:
+          if ($ftype == TType::STRUCT) {
+            $this->timestampStats = new \metastore\TimestampColumnStatsData();
+            $xfer += $this->timestampStats->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -10097,6 +10369,14 @@ class ColumnStatisticsData {
       $xfer += $this->dateStats->write($output);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->timestampStats !== null) {
+      if (!is_object($this->timestampStats)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('timestampStats', TType::STRUCT, 8);
+      $xfer += $this->timestampStats->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index d4fefff..89adea2 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -6795,6 +6795,196 @@ class DateColumnStatsData:
   def __ne__(self, other):
     return not (self == other)
 
+class Timestamp:
+  """
+  Attributes:
+   - secondsSinceEpoch
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I64, 'secondsSinceEpoch', None, None, ), # 1
+  )
+
+  def __init__(self, secondsSinceEpoch=None,):
+    self.secondsSinceEpoch = secondsSinceEpoch
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I64:
+          self.secondsSinceEpoch = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('Timestamp')
+    if self.secondsSinceEpoch is not None:
+      oprot.writeFieldBegin('secondsSinceEpoch', TType.I64, 1)
+      oprot.writeI64(self.secondsSinceEpoch)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.secondsSinceEpoch is None:
+      raise TProtocol.TProtocolException(message='Required field secondsSinceEpoch is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.secondsSinceEpoch)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class TimestampColumnStatsData:
+  """
+  Attributes:
+   - lowValue
+   - highValue
+   - numNulls
+   - numDVs
+   - bitVectors
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'lowValue', (Timestamp, Timestamp.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'highValue', (Timestamp, Timestamp.thrift_spec), None, ), # 2
+    (3, TType.I64, 'numNulls', None, None, ), # 3
+    (4, TType.I64, 'numDVs', None, None, ), # 4
+    (5, TType.STRING, 'bitVectors', None, None, ), # 5
+  )
+
+  def __init__(self, lowValue=None, highValue=None, numNulls=None, numDVs=None, bitVectors=None,):
+    self.lowValue = lowValue
+    self.highValue = highValue
+    self.numNulls = numNulls
+    self.numDVs = numDVs
+    self.bitVectors = bitVectors
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.lowValue = Timestamp()
+          self.lowValue.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.highValue = Timestamp()
+          self.highValue.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I64:
+          self.numNulls = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.I64:
+          self.numDVs = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.bitVectors = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TimestampColumnStatsData')
+    if self.lowValue is not None:
+      oprot.writeFieldBegin('lowValue', TType.STRUCT, 1)
+      self.lowValue.write(oprot)
+      oprot.writeFieldEnd()
+    if self.highValue is not None:
+      oprot.writeFieldBegin('highValue', TType.STRUCT, 2)
+      self.highValue.write(oprot)
+      oprot.writeFieldEnd()
+    if self.numNulls is not None:
+      oprot.writeFieldBegin('numNulls', TType.I64, 3)
+      oprot.writeI64(self.numNulls)
+      oprot.writeFieldEnd()
+    if self.numDVs is not None:
+      oprot.writeFieldBegin('numDVs', TType.I64, 4)
+      oprot.writeI64(self.numDVs)
+      oprot.writeFieldEnd()
+    if self.bitVectors is not None:
+      oprot.writeFieldBegin('bitVectors', TType.STRING, 5)
+      oprot.writeString(self.bitVectors)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.numNulls is None:
+      raise TProtocol.TProtocolException(message='Required field numNulls is unset!')
+    if self.numDVs is None:
+      raise TProtocol.TProtocolException(message='Required field numDVs is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.lowValue)
+    value = (value * 31) ^ hash(self.highValue)
+    value = (value * 31) ^ hash(self.numNulls)
+    value = (value * 31) ^ hash(self.numDVs)
+    value = (value * 31) ^ hash(self.bitVectors)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class ColumnStatisticsData:
   """
   Attributes:
@@ -6805,6 +6995,7 @@ class ColumnStatisticsData:
    - binaryStats
    - decimalStats
    - dateStats
+   - timestampStats
   """
 
   thrift_spec = (
@@ -6816,9 +7007,10 @@ class ColumnStatisticsData:
     (5, TType.STRUCT, 'binaryStats', (BinaryColumnStatsData, BinaryColumnStatsData.thrift_spec), None, ), # 5
     (6, TType.STRUCT, 'decimalStats', (DecimalColumnStatsData, DecimalColumnStatsData.thrift_spec), None, ), # 6
     (7, TType.STRUCT, 'dateStats', (DateColumnStatsData, DateColumnStatsData.thrift_spec), None, ), # 7
+    (8, TType.STRUCT, 'timestampStats', (TimestampColumnStatsData, TimestampColumnStatsData.thrift_spec), None, ), # 8
   )
 
-  def __init__(self, booleanStats=None, longStats=None, doubleStats=None, stringStats=None, binaryStats=None, decimalStats=None, dateStats=None,):
+  def __init__(self, booleanStats=None, longStats=None, doubleStats=None, stringStats=None, binaryStats=None, decimalStats=None, dateStats=None, timestampStats=None,):
     self.booleanStats = booleanStats
     self.longStats = longStats
     self.doubleStats = doubleStats
@@ -6826,6 +7018,7 @@ class ColumnStatisticsData:
     self.binaryStats = binaryStats
     self.decimalStats = decimalStats
     self.dateStats = dateStats
+    self.timestampStats = timestampStats
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -6878,6 +7071,12 @@ class ColumnStatisticsData:
           self.dateStats.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.STRUCT:
+          self.timestampStats = TimestampColumnStatsData()
+          self.timestampStats.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -6916,6 +7115,10 @@ class ColumnStatisticsData:
       oprot.writeFieldBegin('dateStats', TType.STRUCT, 7)
       self.dateStats.write(oprot)
       oprot.writeFieldEnd()
+    if self.timestampStats is not None:
+      oprot.writeFieldBegin('timestampStats', TType.STRUCT, 8)
+      self.timestampStats.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -6932,6 +7135,7 @@ class ColumnStatisticsData:
     value = (value * 31) ^ hash(self.binaryStats)
     value = (value * 31) ^ hash(self.decimalStats)
     value = (value * 31) ^ hash(self.dateStats)
+    value = (value * 31) ^ hash(self.timestampStats)
     return value
 
   def __repr__(self):
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 4863de6..13b05bd 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -1510,6 +1510,49 @@ class DateColumnStatsData
   ::Thrift::Struct.generate_accessors self
 end
 
+class Timestamp
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  SECONDSSINCEEPOCH = 1
+
+  FIELDS = {
+    SECONDSSINCEEPOCH => {:type => ::Thrift::Types::I64, :name => 'secondsSinceEpoch'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field secondsSinceEpoch is unset!') unless @secondsSinceEpoch
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class TimestampColumnStatsData
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  LOWVALUE = 1
+  HIGHVALUE = 2
+  NUMNULLS = 3
+  NUMDVS = 4
+  BITVECTORS = 5
+
+  FIELDS = {
+    LOWVALUE => {:type => ::Thrift::Types::STRUCT, :name => 'lowValue', :class => ::Timestamp, :optional => true},
+    HIGHVALUE => {:type => ::Thrift::Types::STRUCT, :name => 'highValue', :class => ::Timestamp, :optional => true},
+    NUMNULLS => {:type => ::Thrift::Types::I64, :name => 'numNulls'},
+    NUMDVS => {:type => ::Thrift::Types::I64, :name => 'numDVs'},
+    BITVECTORS => {:type => ::Thrift::Types::STRING, :name => 'bitVectors', :binary => true, :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field numNulls is unset!') unless @numNulls
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field numDVs is unset!') unless @numDVs
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class ColumnStatisticsData < ::Thrift::Union
   include ::Thrift::Struct_Union
   class << self
@@ -1540,6 +1583,10 @@ class ColumnStatisticsData < ::Thrift::Union
     def dateStats(val)
       ColumnStatisticsData.new(:dateStats, val)
     end
+
+    def timestampStats(val)
+      ColumnStatisticsData.new(:timestampStats, val)
+    end
   end
 
   BOOLEANSTATS = 1
@@ -1549,6 +1596,7 @@ class ColumnStatisticsData < ::Thrift::Union
   BINARYSTATS = 5
   DECIMALSTATS = 6
   DATESTATS = 7
+  TIMESTAMPSTATS = 8
 
   FIELDS = {
     BOOLEANSTATS => {:type => ::Thrift::Types::STRUCT, :name => 'booleanStats', :class => ::BooleanColumnStatsData},
@@ -1557,7 +1605,8 @@ class ColumnStatisticsData < ::Thrift::Union
     STRINGSTATS => {:type => ::Thrift::Types::STRUCT, :name => 'stringStats', :class => ::StringColumnStatsData},
     BINARYSTATS => {:type => ::Thrift::Types::STRUCT, :name => 'binaryStats', :class => ::BinaryColumnStatsData},
     DECIMALSTATS => {:type => ::Thrift::Types::STRUCT, :name => 'decimalStats', :class => ::DecimalColumnStatsData},
-    DATESTATS => {:type => ::Thrift::Types::STRUCT, :name => 'dateStats', :class => ::DateColumnStatsData}
+    DATESTATS => {:type => ::Thrift::Types::STRUCT, :name => 'dateStats', :class => ::DateColumnStatsData},
+    TIMESTAMPSTATS => {:type => ::Thrift::Types::STRUCT, :name => 'timestampStats', :class => ::TimestampColumnStatsData}
   }
 
   def struct_fields; FIELDS; end
diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
index 449393c..7b5b5fb 100644
--- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
@@ -562,6 +562,18 @@ struct DateColumnStatsData {
 5: optional binary bitVectors
 }
 
+struct Timestamp {
+1: required i64 secondsSinceEpoch
+}
+
+struct TimestampColumnStatsData {
+1: optional Timestamp lowValue,
+2: optional Timestamp highValue,
+3: required i64 numNulls,
+4: required i64 numDVs,
+5: optional binary bitVectors
+}
+
 union ColumnStatisticsData {
 1: BooleanColumnStatsData booleanStats,
 2: LongColumnStatsData longStats,
@@ -569,7 +581,8 @@ union ColumnStatisticsData {
 4: StringColumnStatsData stringStats,
 5: BinaryColumnStatsData binaryStats,
 6: DecimalColumnStatsData decimalStats,
-7: DateColumnStatsData dateStats
+7: DateColumnStatsData dateStats,
+8: TimestampColumnStatsData timestampStats
 }
 
 struct ColumnStatisticsObj {
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
index e8d197a..bc177e7 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
@@ -37,12 +37,15 @@ import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.Timestamp;
+import org.apache.hadoop.hive.metastore.api.TimestampColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.utils.DecimalUtils;
 import org.apache.hadoop.hive.metastore.columnstats.cache.DateColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.DecimalColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.DoubleColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.LongColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.StringColumnStatsDataInspector;
+import org.apache.hadoop.hive.metastore.columnstats.cache.TimestampColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.model.MPartition;
 import org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics;
 import org.apache.hadoop.hive.metastore.model.MTable;
@@ -126,6 +129,14 @@ public class StatObjectConverter {
            dateStats.isSetBitVectors() ? dateStats.getBitVectors() : null,
            dateStats.isSetLowValue() ? dateStats.getLowValue().getDaysSinceEpoch() : null,
            dateStats.isSetHighValue() ? dateStats.getHighValue().getDaysSinceEpoch() : null);
+     } else if (statsObj.getStatsData().isSetTimestampStats()) {
+       TimestampColumnStatsData timestampStats = statsObj.getStatsData().getTimestampStats();
+       mColStats.setTimestampStats(
+           timestampStats.isSetNumNulls() ? timestampStats.getNumNulls() : null,
+           timestampStats.isSetNumDVs() ? timestampStats.getNumDVs() : null,
+           timestampStats.isSetBitVectors() ? timestampStats.getBitVectors() : null,
+           timestampStats.isSetLowValue() ? timestampStats.getLowValue().getSecondsSinceEpoch() : null,
+           timestampStats.isSetHighValue() ? timestampStats.getHighValue().getSecondsSinceEpoch() : null);
      }
      mColStats.setEngine(engine);
      return mColStats;
@@ -251,8 +262,7 @@ public class StatObjectConverter {
       binaryStats.setMaxColLen(mStatsObj.getMaxColLen());
       colStatsData.setBinaryStats(binaryStats);
     } else if (colType.equals("bigint") || colType.equals("int") ||
-        colType.equals("smallint") || colType.equals("tinyint") ||
-        colType.equals("timestamp")) {
+        colType.equals("smallint") || colType.equals("tinyint")) {
       LongColumnStatsDataInspector longStats = new LongColumnStatsDataInspector();
       longStats.setNumNulls(mStatsObj.getNumNulls());
       Long longHighValue = mStatsObj.getLongHighValue();
@@ -308,6 +318,20 @@ public class StatObjectConverter {
       dateStats.setNumDVs(mStatsObj.getNumDVs());
       dateStats.setBitVectors((mStatsObj.getBitVector()==null||!enableBitVector)? null : mStatsObj.getBitVector());
       colStatsData.setDateStats(dateStats);
+    } else if (colType.equals("timestamp")) {
+      TimestampColumnStatsDataInspector timestampStats = new TimestampColumnStatsDataInspector();
+      timestampStats.setNumNulls(mStatsObj.getNumNulls());
+      Long highValue = mStatsObj.getLongHighValue();
+      if (highValue != null) {
+        timestampStats.setHighValue(new Timestamp(highValue));
+      }
+      Long lowValue = mStatsObj.getLongLowValue();
+      if (lowValue != null) {
+        timestampStats.setLowValue(new Timestamp(lowValue));
+      }
+      timestampStats.setNumDVs(mStatsObj.getNumDVs());
+      timestampStats.setBitVectors((mStatsObj.getBitVector()==null||!enableBitVector)? null : mStatsObj.getBitVector());
+      colStatsData.setTimestampStats(timestampStats);
     }
     statsObj.setStatsData(colStatsData);
     return statsObj;
@@ -394,6 +418,14 @@ public class StatObjectConverter {
           dateStats.isSetBitVectors() ? dateStats.getBitVectors() : null,
           dateStats.isSetLowValue() ? dateStats.getLowValue().getDaysSinceEpoch() : null,
           dateStats.isSetHighValue() ? dateStats.getHighValue().getDaysSinceEpoch() : null);
+    } else if (statsObj.getStatsData().isSetTimestampStats()) {
+      TimestampColumnStatsData timestampStats = statsObj.getStatsData().getTimestampStats();
+      mColStats.setTimestampStats(
+          timestampStats.isSetNumNulls() ? timestampStats.getNumNulls() : null,
+          timestampStats.isSetNumDVs() ? timestampStats.getNumDVs() : null,
+          timestampStats.isSetBitVectors() ? timestampStats.getBitVectors() : null,
+          timestampStats.isSetLowValue() ? timestampStats.getLowValue().getSecondsSinceEpoch() : null,
+          timestampStats.isSetHighValue() ? timestampStats.getHighValue().getSecondsSinceEpoch() : null);
     }
     mColStats.setEngine(engine);
     return mColStats;
@@ -429,8 +461,7 @@ public class StatObjectConverter {
       binaryStats.setMaxColLen(mStatsObj.getMaxColLen());
       colStatsData.setBinaryStats(binaryStats);
     } else if (colType.equals("tinyint") || colType.equals("smallint") ||
-        colType.equals("int") || colType.equals("bigint") ||
-        colType.equals("timestamp")) {
+        colType.equals("int") || colType.equals("bigint")) {
       LongColumnStatsDataInspector longStats = new LongColumnStatsDataInspector();
       longStats.setNumNulls(mStatsObj.getNumNulls());
       if (mStatsObj.getLongHighValue() != null) {
@@ -480,6 +511,20 @@ public class StatObjectConverter {
       dateStats.setNumDVs(mStatsObj.getNumDVs());
       dateStats.setBitVectors((mStatsObj.getBitVector()==null||!enableBitVector)? null : mStatsObj.getBitVector());
       colStatsData.setDateStats(dateStats);
+    } else if (colType.equals("timestamp")) {
+      TimestampColumnStatsDataInspector timestampStats = new TimestampColumnStatsDataInspector();
+      timestampStats.setNumNulls(mStatsObj.getNumNulls());
+      Long highValue = mStatsObj.getLongHighValue();
+      if (highValue != null) {
+        timestampStats.setHighValue(new Timestamp(highValue));
+      }
+      Long lowValue = mStatsObj.getLongLowValue();
+      if (lowValue != null) {
+        timestampStats.setLowValue(new Timestamp(lowValue));
+      }
+      timestampStats.setNumDVs(mStatsObj.getNumDVs());
+      timestampStats.setBitVectors((mStatsObj.getBitVector()==null||!enableBitVector)? null : mStatsObj.getBitVector());
+      colStatsData.setTimestampStats(timestampStats);
     }
     statsObj.setStatsData(colStatsData);
     return statsObj;
@@ -524,8 +569,7 @@ public class StatObjectConverter {
       binaryStats.setMaxColLen(MetastoreDirectSqlUtils.extractSqlLong(maxlen));
       data.setBinaryStats(binaryStats);
     } else if (colType.equals("bigint") || colType.equals("int") ||
-        colType.equals("smallint") || colType.equals("tinyint") ||
-        colType.equals("timestamp")) {
+        colType.equals("smallint") || colType.equals("tinyint")) {
       LongColumnStatsDataInspector longStats = new LongColumnStatsDataInspector();
       longStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       if (lhigh != null) {
@@ -573,6 +617,18 @@ public class StatObjectConverter {
       dateStats.setNumDVs(MetastoreDirectSqlUtils.extractSqlLong(dist));
       dateStats.setBitVectors(MetastoreDirectSqlUtils.extractSqlBlob(bitVector));
       data.setDateStats(dateStats);
+    } else if (colType.equals("timestamp")) {
+      TimestampColumnStatsDataInspector timestampStats = new TimestampColumnStatsDataInspector();
+      timestampStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
+      if (lhigh != null) {
+        timestampStats.setHighValue(new Timestamp(MetastoreDirectSqlUtils.extractSqlLong(lhigh)));
+      }
+      if (llow != null) {
+        timestampStats.setLowValue(new Timestamp(MetastoreDirectSqlUtils.extractSqlLong(llow)));
+      }
+      timestampStats.setNumDVs(MetastoreDirectSqlUtils.extractSqlLong(dist));
+      timestampStats.setBitVectors(MetastoreDirectSqlUtils.extractSqlBlob(bitVector));
+      data.setTimestampStats(timestampStats);
     }
   }
 
@@ -604,7 +660,7 @@ public class StatObjectConverter {
       binaryStats.setMaxColLen(MetastoreDirectSqlUtils.extractSqlLong(maxlen));
       data.setBinaryStats(binaryStats);
     } else if (colType.equals("bigint") || colType.equals("int") || colType.equals("smallint")
-        || colType.equals("tinyint") || colType.equals("timestamp")) {
+        || colType.equals("tinyint")) {
       LongColumnStatsDataInspector longStats = new LongColumnStatsDataInspector();
       longStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       if (lhigh != null) {
@@ -674,6 +730,41 @@ public class StatObjectConverter {
       estimation = Math.min(estimation, rangeBound);
       dateStats.setNumDVs(estimation);
       data.setDateStats(dateStats);
+    } else if (colType.equals("timestamp")) {
+      TimestampColumnStatsDataInspector timestampStats = new TimestampColumnStatsDataInspector();
+      timestampStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
+      if (lhigh != null) {
+        timestampStats.setHighValue(new Timestamp(MetastoreDirectSqlUtils.extractSqlLong(lhigh)));
+      }
+      if (llow != null) {
+        timestampStats.setLowValue(new Timestamp(MetastoreDirectSqlUtils.extractSqlLong(llow)));
+      }
+      long lowerBound = MetastoreDirectSqlUtils.extractSqlLong(dist);
+      long higherBound = MetastoreDirectSqlUtils.extractSqlLong(sumDist);
+      long rangeBound = Long.MAX_VALUE;
+      if (lhigh != null && llow != null) {
+        rangeBound = MetastoreDirectSqlUtils.extractSqlLong(lhigh)
+            - MetastoreDirectSqlUtils.extractSqlLong(llow) + 1;
+      }
+      long estimation;
+      if (useDensityFunctionForNDVEstimation && lhigh != null && llow != null && avgLong != null
+          && MetastoreDirectSqlUtils.extractSqlDouble(avgLong) != 0.0) {
+        // We have estimation, lowerbound and higherbound. We use estimation if
+        // it is between lowerbound and higherbound.
+        estimation = MetastoreDirectSqlUtils
+            .extractSqlLong((MetastoreDirectSqlUtils.extractSqlLong(lhigh) - MetastoreDirectSqlUtils
+                .extractSqlLong(llow)) / MetastoreDirectSqlUtils.extractSqlDouble(avgLong));
+        if (estimation < lowerBound) {
+          estimation = lowerBound;
+        } else if (estimation > higherBound) {
+          estimation = higherBound;
+        }
+      } else {
+        estimation = (long) (lowerBound + (higherBound - lowerBound) * ndvTuner);
+      }
+      estimation = Math.min(estimation, rangeBound);
+      timestampStats.setNumDVs(estimation);
+      data.setTimestampStats(timestampStats);
     } else if (colType.equals("double") || colType.equals("float")) {
       DoubleColumnStatsDataInspector doubleStats = new DoubleColumnStatsDataInspector();
       doubleStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
@@ -888,6 +979,26 @@ public class StatObjectConverter {
       }
       break;
     }
+    case TIMESTAMP_STATS: {
+      TimestampColumnStatsData oldTimestampStatsData = oldStatObj.getStatsData().getTimestampStats();
+      TimestampColumnStatsData newTimestampStatsData = newStatObj.getStatsData().getTimestampStats();
+      if (newTimestampStatsData.isSetHighValue()) {
+        oldTimestampStatsData.setHighValue(newTimestampStatsData.getHighValue());
+      }
+      if (newTimestampStatsData.isSetLowValue()) {
+        oldTimestampStatsData.setLowValue(newTimestampStatsData.getLowValue());
+      }
+      if (newTimestampStatsData.isSetNumNulls()) {
+        oldTimestampStatsData.setNumNulls(newTimestampStatsData.getNumNulls());
+      }
+      if (newTimestampStatsData.isSetNumDVs()) {
+        oldTimestampStatsData.setNumDVs(newTimestampStatsData.getNumDVs());
+      }
+      if (newTimestampStatsData.isSetBitVectors()) {
+        oldTimestampStatsData.setBitVectors(newTimestampStatsData.getBitVectors());
+      }
+      break;
+    }
     default:
       throw new IllegalArgumentException("Unknown stats type: " + typeNew.toString());
     }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/ColumnsStatsUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/ColumnsStatsUtils.java
index 2d6d226..dee750d 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/ColumnsStatsUtils.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/ColumnsStatsUtils.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.metastore.columnstats.cache.DateColumnStatsDataIns
 import org.apache.hadoop.hive.metastore.columnstats.cache.DecimalColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.DoubleColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.LongColumnStatsDataInspector;
+import org.apache.hadoop.hive.metastore.columnstats.cache.TimestampColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.StringColumnStatsDataInspector;
 
 /**
@@ -49,6 +50,22 @@ public final class ColumnsStatsUtils {
   }
 
   /**
+   * Convertes to TimestampColumnStatsDataInspector if it's a TimestampColumnStatsData.
+   * @param cso ColumnStatisticsObj
+   * @return TimestampColumnStatsDataInspector
+   */
+  public static TimestampColumnStatsDataInspector timestampInspectorFromStats(ColumnStatisticsObj cso) {
+    TimestampColumnStatsDataInspector timestampColumnStats;
+    if (cso.getStatsData().getTimestampStats() instanceof TimestampColumnStatsDataInspector) {
+      timestampColumnStats =
+          (TimestampColumnStatsDataInspector)(cso.getStatsData().getTimestampStats());
+    } else {
+      timestampColumnStats = new TimestampColumnStatsDataInspector(cso.getStatsData().getTimestampStats());
+    }
+    return timestampColumnStats;
+  }
+
+  /**
    * Convertes to StringColumnStatsDataInspector
    * if it's a StringColumnStatsData.
    * @param cso ColumnStatisticsObj
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/aggr/ColumnStatsAggregatorFactory.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/aggr/ColumnStatsAggregatorFactory.java
index 7aaab4a..c2d14df 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/aggr/ColumnStatsAggregatorFactory.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/aggr/ColumnStatsAggregatorFactory.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.metastore.columnstats.cache.DecimalColumnStatsData
 import org.apache.hadoop.hive.metastore.columnstats.cache.DoubleColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.LongColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.StringColumnStatsDataInspector;
+import org.apache.hadoop.hive.metastore.columnstats.cache.TimestampColumnStatsDataInspector;
 
 public class ColumnStatsAggregatorFactory {
 
@@ -48,6 +49,9 @@ public class ColumnStatsAggregatorFactory {
     case DATE_STATS:
       agg = new DateColumnStatsAggregator();
       break;
+    case TIMESTAMP_STATS:
+      agg = new TimestampColumnStatsAggregator();
+      break;
     case DOUBLE_STATS:
       agg = new DoubleColumnStatsAggregator();
       break;
@@ -86,6 +90,10 @@ public class ColumnStatsAggregatorFactory {
       csd.setDateStats(new DateColumnStatsDataInspector());
       break;
 
+    case TIMESTAMP_STATS:
+      csd.setTimestampStats(new TimestampColumnStatsDataInspector());
+      break;
+
     case DOUBLE_STATS:
       csd.setDoubleStats(new DoubleColumnStatsDataInspector());
       break;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/aggr/TimestampColumnStatsAggregator.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/aggr/TimestampColumnStatsAggregator.java
new file mode 100644
index 0000000..8828f89
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/aggr/TimestampColumnStatsAggregator.java
@@ -0,0 +1,362 @@
+/*
+ * 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.metastore.columnstats.aggr;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hive.common.ndv.NumDistinctValueEstimator;
+import org.apache.hadoop.hive.common.ndv.NumDistinctValueEstimatorFactory;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.Timestamp;
+import org.apache.hadoop.hive.metastore.api.TimestampColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.columnstats.cache.TimestampColumnStatsDataInspector;
+import org.apache.hadoop.hive.metastore.columnstats.merge.TimestampColumnStatsMerger;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils.ColStatsObjWithSourceInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.hive.metastore.columnstats.ColumnsStatsUtils.timestampInspectorFromStats;
+
+public class TimestampColumnStatsAggregator extends ColumnStatsAggregator implements
+    IExtrapolatePartStatus {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TimestampColumnStatsAggregator.class);
+
+  @Override
+  public ColumnStatisticsObj aggregate(List<ColStatsObjWithSourceInfo> colStatsWithSourceInfo,
+                                       List<String> partNames, boolean areAllPartsFound) throws MetaException {
+    ColumnStatisticsObj statsObj = null;
+    String colType = null;
+    String colName = null;
+    // check if all the ColumnStatisticsObjs contain stats and all the ndv are
+    // bitvectors
+    boolean doAllPartitionContainStats = partNames.size() == colStatsWithSourceInfo.size();
+    NumDistinctValueEstimator ndvEstimator = null;
+    for (ColStatsObjWithSourceInfo csp : colStatsWithSourceInfo) {
+      ColumnStatisticsObj cso = csp.getColStatsObj();
+      if (statsObj == null) {
+        colName = cso.getColName();
+        colType = cso.getColType();
+        statsObj = ColumnStatsAggregatorFactory.newColumnStaticsObj(colName, colType,
+            cso.getStatsData().getSetField());
+        LOG.trace("doAllPartitionContainStats for column: {} is: {}", colName, doAllPartitionContainStats);
+      }
+      TimestampColumnStatsDataInspector timestampColumnStats = timestampInspectorFromStats(cso);
+
+      if (timestampColumnStats.getNdvEstimator() == null) {
+        ndvEstimator = null;
+        break;
+      } else {
+        // check if all of the bit vectors can merge
+        NumDistinctValueEstimator estimator = timestampColumnStats.getNdvEstimator();
+        if (ndvEstimator == null) {
+          ndvEstimator = estimator;
+        } else {
+          if (ndvEstimator.canMerge(estimator)) {
+            continue;
+          } else {
+            ndvEstimator = null;
+            break;
+          }
+        }
+      }
+    }
+    if (ndvEstimator != null) {
+      ndvEstimator = NumDistinctValueEstimatorFactory
+          .getEmptyNumDistinctValueEstimator(ndvEstimator);
+    }
+    LOG.debug("all of the bit vectors can merge for " + colName + " is " + (ndvEstimator != null));
+    ColumnStatisticsData columnStatisticsData = new ColumnStatisticsData();
+    if (doAllPartitionContainStats || colStatsWithSourceInfo.size() < 2) {
+      TimestampColumnStatsDataInspector aggregateData = null;
+      long lowerBound = 0;
+      long higherBound = 0;
+      double densityAvgSum = 0.0;
+      for (ColStatsObjWithSourceInfo csp : colStatsWithSourceInfo) {
+        ColumnStatisticsObj cso = csp.getColStatsObj();
+        TimestampColumnStatsDataInspector newData = timestampInspectorFromStats(cso);
+        higherBound += newData.getNumDVs();
+        if (newData.isSetLowValue() && newData.isSetHighValue()) {
+          densityAvgSum += (diff(newData.getHighValue(), newData.getLowValue())) / newData.getNumDVs();
+        }
+        if (ndvEstimator != null) {
+          ndvEstimator.mergeEstimators(newData.getNdvEstimator());
+        }
+        if (aggregateData == null) {
+          aggregateData = newData.deepCopy();
+        } else {
+          TimestampColumnStatsMerger merger = new TimestampColumnStatsMerger();
+          merger.setLowValue(aggregateData, newData);
+          merger.setHighValue(aggregateData, newData);
+
+          aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
+          aggregateData.setNumDVs(Math.max(aggregateData.getNumDVs(), newData.getNumDVs()));
+        }
+      }
+      if (ndvEstimator != null) {
+        // if all the ColumnStatisticsObjs contain bitvectors, we do not need to
+        // use uniform distribution assumption because we can merge bitvectors
+        // to get a good estimation.
+        aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
+      } else {
+        long estimation;
+        if (useDensityFunctionForNDVEstimation) {
+          // We have estimation, lowerbound and higherbound. We use estimation
+          // if it is between lowerbound and higherbound.
+          double densityAvg = densityAvgSum / partNames.size();
+          estimation = (long) (diff(aggregateData.getHighValue(), aggregateData.getLowValue()) / densityAvg);
+          if (estimation < lowerBound) {
+            estimation = lowerBound;
+          } else if (estimation > higherBound) {
+            estimation = higherBound;
+          }
+        } else {
+          estimation = (long) (lowerBound + (higherBound - lowerBound) * ndvTuner);
+        }
+        aggregateData.setNumDVs(estimation);
+      }
+      columnStatisticsData.setTimestampStats(aggregateData);
+    } else {
+      // we need extrapolation
+      LOG.debug("start extrapolation for " + colName);
+
+      Map<String, Integer> indexMap = new HashMap<>();
+      for (int index = 0; index < partNames.size(); index++) {
+        indexMap.put(partNames.get(index), index);
+      }
+      Map<String, Double> adjustedIndexMap = new HashMap<>();
+      Map<String, ColumnStatisticsData> adjustedStatsMap = new HashMap<>();
+      // while we scan the css, we also get the densityAvg, lowerbound and
+      // higerbound when useDensityFunctionForNDVEstimation is true.
+      double densityAvgSum = 0.0;
+      if (ndvEstimator == null) {
+        // if not every partition uses bitvector for ndv, we just fall back to
+        // the traditional extrapolation methods.
+        for (ColStatsObjWithSourceInfo csp : colStatsWithSourceInfo) {
+          ColumnStatisticsObj cso = csp.getColStatsObj();
+          String partName = csp.getPartName();
+          TimestampColumnStatsData newData = cso.getStatsData().getTimestampStats();
+          if (useDensityFunctionForNDVEstimation) {
+            densityAvgSum += diff(newData.getHighValue(), newData.getLowValue()) / newData.getNumDVs();
+          }
+          adjustedIndexMap.put(partName, (double) indexMap.get(partName));
+          adjustedStatsMap.put(partName, cso.getStatsData());
+        }
+      } else {
+        // we first merge all the adjacent bitvectors that we could merge and
+        // derive new partition names and index.
+        StringBuilder pseudoPartName = new StringBuilder();
+        double pseudoIndexSum = 0;
+        int length = 0;
+        int curIndex = -1;
+        TimestampColumnStatsDataInspector aggregateData = null;
+        for (ColStatsObjWithSourceInfo csp : colStatsWithSourceInfo) {
+          ColumnStatisticsObj cso = csp.getColStatsObj();
+          String partName = csp.getPartName();
+          TimestampColumnStatsDataInspector newData = timestampInspectorFromStats(cso);
+          // newData.isSetBitVectors() should be true for sure because we
+          // already checked it before.
+          if (indexMap.get(partName) != curIndex) {
+            // There is bitvector, but it is not adjacent to the previous ones.
+            if (length > 0) {
+              // we have to set ndv
+              adjustedIndexMap.put(pseudoPartName.toString(), pseudoIndexSum / length);
+              aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
+              ColumnStatisticsData csd = new ColumnStatisticsData();
+              csd.setTimestampStats(aggregateData);
+              adjustedStatsMap.put(pseudoPartName.toString(), csd);
+              if (useDensityFunctionForNDVEstimation) {
+                densityAvgSum += diff(aggregateData.getHighValue(), aggregateData.getLowValue())
+                    / aggregateData.getNumDVs();
+              }
+              // reset everything
+              pseudoPartName = new StringBuilder();
+              pseudoIndexSum = 0;
+              length = 0;
+              ndvEstimator = NumDistinctValueEstimatorFactory.getEmptyNumDistinctValueEstimator(ndvEstimator);
+            }
+            aggregateData = null;
+          }
+          curIndex = indexMap.get(partName);
+          pseudoPartName.append(partName);
+          pseudoIndexSum += curIndex;
+          length++;
+          curIndex++;
+          if (aggregateData == null) {
+            aggregateData = newData.deepCopy();
+          } else {
+            aggregateData.setLowValue(min(aggregateData.getLowValue(), newData.getLowValue()));
+            aggregateData.setHighValue(max(aggregateData.getHighValue(), newData.getHighValue()));
+            aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
+          }
+          ndvEstimator.mergeEstimators(newData.getNdvEstimator());
+        }
+        if (length > 0) {
+          // we have to set ndv
+          adjustedIndexMap.put(pseudoPartName.toString(), pseudoIndexSum / length);
+          aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
+          ColumnStatisticsData csd = new ColumnStatisticsData();
+          csd.setTimestampStats(aggregateData);
+          adjustedStatsMap.put(pseudoPartName.toString(), csd);
+          if (useDensityFunctionForNDVEstimation) {
+            densityAvgSum += diff(aggregateData.getHighValue(), aggregateData.getLowValue())
+                / aggregateData.getNumDVs();
+          }
+        }
+      }
+      extrapolate(columnStatisticsData, partNames.size(), colStatsWithSourceInfo.size(),
+          adjustedIndexMap, adjustedStatsMap, densityAvgSum / adjustedStatsMap.size());
+    }
+    LOG.debug(
+        "Ndv estimatation for {} is {} # of partitions requested: {} # of partitions found: {}",
+        colName, columnStatisticsData.getTimestampStats().getNumDVs(), partNames.size(),
+        colStatsWithSourceInfo.size());
+    statsObj.setStatsData(columnStatisticsData);
+    return statsObj;
+  }
+
+  private long diff(Timestamp d1, Timestamp d2) {
+    return d1.getSecondsSinceEpoch() - d2.getSecondsSinceEpoch();
+  }
+
+  private Timestamp min(Timestamp d1, Timestamp d2) {
+    return d1.compareTo(d2) < 0 ? d1 : d2;
+  }
+
+  private Timestamp max(Timestamp d1, Timestamp d2) {
+    return d1.compareTo(d2) < 0 ? d2 : d1;
+  }
+
+  @Override
+  public void extrapolate(ColumnStatisticsData extrapolateData, int numParts,
+                          int numPartsWithStats, Map<String, Double> adjustedIndexMap,
+                          Map<String, ColumnStatisticsData> adjustedStatsMap, double densityAvg) {
+    int rightBorderInd = numParts;
+    TimestampColumnStatsDataInspector extrapolateTimestampData = new TimestampColumnStatsDataInspector();
+    Map<String, TimestampColumnStatsData> extractedAdjustedStatsMap = new HashMap<>();
+    for (Map.Entry<String, ColumnStatisticsData> entry : adjustedStatsMap.entrySet()) {
+      extractedAdjustedStatsMap.put(entry.getKey(), entry.getValue().getTimestampStats());
+    }
+    List<Map.Entry<String, TimestampColumnStatsData>> list = new LinkedList<>(
+        extractedAdjustedStatsMap.entrySet());
+    // get the lowValue
+    Collections.sort(list, new Comparator<Map.Entry<String, TimestampColumnStatsData>>() {
+      @Override
+      public int compare(Map.Entry<String, TimestampColumnStatsData> o1,
+                         Map.Entry<String, TimestampColumnStatsData> o2) {
+        return o1.getValue().getLowValue().compareTo(o2.getValue().getLowValue());
+      }
+    });
+    double minInd = adjustedIndexMap.get(list.get(0).getKey());
+    double maxInd = adjustedIndexMap.get(list.get(list.size() - 1).getKey());
+    long lowValue = 0;
... 391 lines suppressed ...