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/10/15 21:37:43 UTC

[hive] branch master updated: HIVE-22239: Scale data size using column value ranges (Jesus Camacho Rodriguez, reviewed by Zoltan Haindrich)

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 fbabad1  HIVE-22239: Scale data size using column value ranges (Jesus Camacho Rodriguez, reviewed by Zoltan Haindrich)
fbabad1 is described below

commit fbabad169023f49a9efc67452a0ef76730f4f09d
Author: Jesus Camacho Rodriguez <jc...@apache.org>
AuthorDate: Wed Oct 9 06:11:01 2019 -0700

    HIVE-22239: Scale data size using column value ranges (Jesus Camacho Rodriguez, reviewed by Zoltan Haindrich)
    
    Close apache/hive#787
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |   5 +
 .../stats/annotation/AnnotateStatsProcCtx.java     |  23 +-
 .../stats/annotation/StatsRulesProcFactory.java    | 168 ++++++++---
 .../apache/hadoop/hive/ql/stats/StatsUtils.java    |   6 +-
 .../queries/clientpositive/bucket_map_join_tez2.q  |   1 +
 .../clientpositive/retry_failure_stat_changes.q    |   1 +
 .../queries/clientpositive/semijoin_reddedup.q     |   1 +
 .../clientpositive/annotate_stats_filter.q.out     |  24 +-
 .../clientpositive/annotate_stats_join_pkfk.q.out  |  42 +--
 .../results/clientpositive/cbo_rp_auto_join1.q.out |  28 +-
 .../clientpositive/correlationoptimizer13.q.out    |  38 +--
 .../clientpositive/correlationoptimizer9.q.out     | 124 ++++----
 .../results/clientpositive/folder_predicate.q.out  |  30 +-
 .../clientpositive/fp_literal_arithmetic.q.out     |   8 +-
 .../test/results/clientpositive/join_merging.q.out |  38 +--
 .../llap/acid_vectorization_original.q.out         |  30 +-
 .../clientpositive/llap/auto_smb_mapjoin_14.q.out  |  94 +++---
 .../llap/auto_sortmerge_join_10.q.out              |  34 +--
 .../llap/auto_sortmerge_join_9.q.out               | 212 +++++++-------
 .../llap/colstats_date_min_max.q.out               |   4 +-
 .../llap/constraints_optimization.q.out            |  28 +-
 .../llap/dynpart_sort_optimization.q.out           |  22 +-
 .../llap/dynpart_sort_optimization2.q.out          |  38 +--
 .../clientpositive/llap/explainuser_1.q.out        |  28 +-
 .../clientpositive/llap/explainuser_4.q.out        |  38 +--
 .../llap/materialized_view_create_rewrite_2.q.out  |  38 +--
 .../llap/materialized_view_rewrite_1.q.out         |  24 +-
 .../materialized_view_rewrite_no_join_opt_2.q.out  |  24 +-
 .../llap/materialized_view_rewrite_part_1.q.out    |  24 +-
 .../results/clientpositive/llap/orc_llap.q.out     |  24 +-
 .../llap/orc_predicate_pushdown.q.out              |  44 +--
 .../llap/parquet_predicate_pushdown.q.out          |  40 +--
 .../llap/results_cache_with_masking.q.out          |  24 +-
 .../llap/retry_failure_reorder.q.out               |  32 +--
 .../llap/retry_failure_stat_changes.q.out          |  46 +--
 .../clientpositive/llap/runtime_stats_hs2.q.out    |  12 +-
 .../results/clientpositive/llap/semijoin.q.out     |  64 ++---
 .../clientpositive/llap/smb_mapjoin_14.q.out       | 106 +++----
 .../results/clientpositive/llap/subquery_in.q.out  |  20 +-
 .../clientpositive/llap/subquery_select.q.out      |   6 +-
 .../llap/tez_dynpart_hashjoin_1.q.out              |   2 +-
 .../llap/tez_dynpart_hashjoin_2.q.out              |  54 ++--
 .../llap/tez_vector_dynpart_hashjoin_1.q.out       |   2 +-
 .../llap/tez_vector_dynpart_hashjoin_2.q.out       |  54 ++--
 .../clientpositive/llap/vector_between_in.q.out    |  34 +--
 .../llap/vector_decimal_expressions.q.out          |  24 +-
 .../results/clientpositive/llap/vector_elt.q.out   |   4 +-
 .../clientpositive/llap/vector_inner_join.q.out    | 136 ++++-----
 .../llap/vector_leftsemi_mapjoin.q.out             | 320 ++++++++++-----------
 .../clientpositive/llap/vectorization_10.q.out     |   6 +-
 .../clientpositive/llap/vectorization_13.q.out     |  32 +--
 .../clientpositive/llap/vectorization_14.q.out     |  18 +-
 .../clientpositive/llap/vectorization_16.q.out     |  14 +-
 .../clientpositive/llap/vectorization_17.q.out     |  10 +-
 .../clientpositive/llap/vectorization_7.q.out      |  16 +-
 .../clientpositive/llap/vectorization_9.q.out      |  14 +-
 .../clientpositive/llap/vectorization_div0.q.out   |  28 +-
 .../clientpositive/llap/vectorization_limit.q.out  |   8 +-
 .../llap/vectorization_short_regress.q.out         |  42 +--
 ql/src/test/results/clientpositive/masking_1.q.out |  48 ++--
 .../test/results/clientpositive/masking_12.q.out   |  30 +-
 .../test/results/clientpositive/masking_13.q.out   |  12 +-
 .../results/clientpositive/masking_1_newdb.q.out   |  12 +-
 ql/src/test/results/clientpositive/masking_5.q.out |  18 +-
 ql/src/test/results/clientpositive/masking_8.q.out |  24 +-
 .../clientpositive/masking_disablecbo_1.q.out      |  48 ++--
 .../test/results/clientpositive/masking_mv.q.out   |  64 ++---
 .../nullability_transitive_inference.q.out         |   6 +-
 .../clientpositive/parquet_vectorization_10.q.out  |   6 +-
 .../clientpositive/parquet_vectorization_13.q.out  |  32 +--
 .../clientpositive/parquet_vectorization_14.q.out  |  18 +-
 .../clientpositive/parquet_vectorization_16.q.out  |  14 +-
 .../clientpositive/parquet_vectorization_17.q.out  |  10 +-
 .../clientpositive/parquet_vectorization_7.q.out   |  16 +-
 .../clientpositive/parquet_vectorization_9.q.out   |  14 +-
 .../parquet_vectorization_div0.q.out               |  20 +-
 .../parquet_vectorization_limit.q.out              |   4 +-
 .../parquet_vectorization_offset_limit.q.out       |   4 +-
 ql/src/test/results/clientpositive/pcr.q.out       |  60 ++--
 .../perf/tez/constraints/query34.q.out             |  14 +-
 .../perf/tez/constraints/query35.q.out             |  44 +--
 .../perf/tez/constraints/query37.q.out             |  18 +-
 .../perf/tez/constraints/query66.q.out             |  26 +-
 .../perf/tez/constraints/query73.q.out             |  14 +-
 .../perf/tez/constraints/query79.q.out             |  18 +-
 .../perf/tez/constraints/query82.q.out             |  10 +-
 .../perf/tez/constraints/query84.q.out             |  12 +-
 .../perf/tez/constraints/query88.q.out             | 100 +++----
 .../perf/tez/constraints/query90.q.out             |  14 +-
 .../perf/tez/constraints/query96.q.out             |  12 +-
 .../results/clientpositive/perf/tez/query34.q.out  |  14 +-
 .../results/clientpositive/perf/tez/query35.q.out  | 206 ++++++-------
 .../results/clientpositive/perf/tez/query37.q.out  |  18 +-
 .../results/clientpositive/perf/tez/query66.q.out  |  26 +-
 .../results/clientpositive/perf/tez/query73.q.out  |  14 +-
 .../results/clientpositive/perf/tez/query79.q.out  |  18 +-
 .../results/clientpositive/perf/tez/query82.q.out  |  10 +-
 .../results/clientpositive/perf/tez/query84.q.out  |  14 +-
 .../results/clientpositive/perf/tez/query88.q.out  | 100 +++----
 .../results/clientpositive/perf/tez/query90.q.out  |  14 +-
 .../results/clientpositive/perf/tez/query96.q.out  |  12 +-
 .../clientpositive/remove_exprs_stats.q.out        |  18 +-
 ql/src/test/results/clientpositive/sample7.q.out   |   8 +-
 .../spark/dynpart_sort_optimization.q.out          |  14 +-
 .../clientpositive/spark/spark_explainuser_1.q.out |  28 +-
 .../tez/acid_vectorization_original_tez.q.out      |  30 +-
 .../clientpositive/tez/explainanalyze_4.q.out      |  36 +--
 .../tez/vector_non_string_partition.q.out          |  16 +-
 .../results/clientpositive/union_pos_alias.q.out   |  50 ++--
 .../vector_decimal_expressions.q.out               |  24 +-
 .../test/results/clientpositive/vector_elt.q.out   |   4 +-
 .../vector_non_string_partition.q.out              |  16 +-
 .../results/clientpositive/vectorization_10.q.out  |   6 +-
 .../results/clientpositive/vectorization_13.q.out  |  32 +--
 .../results/clientpositive/vectorization_14.q.out  |  18 +-
 .../results/clientpositive/vectorization_16.q.out  |  14 +-
 .../results/clientpositive/vectorization_17.q.out  |  10 +-
 .../results/clientpositive/vectorization_7.q.out   |  16 +-
 .../results/clientpositive/vectorization_9.q.out   |  14 +-
 .../clientpositive/vectorization_limit.q.out       |   8 +-
 .../vectorization_offset_limit.q.out               |   4 +-
 121 files changed, 2027 insertions(+), 1938 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 93536f2..c5e46e9 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2537,6 +2537,11 @@ public class HiveConf extends Configuration {
         "When estimating output rows for a join involving multiple columns, the default behavior assumes" +
         "the columns are independent. Setting this flag to true will cause the estimator to assume" +
         "the columns are correlated."),
+    HIVE_STATS_RANGE_SELECTIVITY_UNIFORM_DISTRIBUTION("hive.stats.filter.range.uniform", true,
+        "When estimating output rows from a condition, if a range predicate is applied over a column and the\n" +
+        "minimum and maximum values for that column are available, assume uniform distribution of values\n" +
+        "across that range and scales number of rows proportionally. If this is set to false, default\n" +
+        "selectivity value is used."),
     // in the absence of uncompressed/raw data size, total file size will be used for statistics
     // annotation. But the file may be compressed, encoded and serialized which may be lesser in size
     // than the actual uncompressed/raw data size. This factor will be multiplied to file size to estimate
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateStatsProcCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateStatsProcCtx.java
index 187ec25..8056161 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateStatsProcCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateStatsProcCtx.java
@@ -31,36 +31,37 @@ public class AnnotateStatsProcCtx implements NodeProcessorCtx {
 
   private ParseContext pctx;
   private HiveConf conf;
-  private Statistics andExprStats = null;
+  private boolean uniformWithinRange;
+  private Statistics andExprStats;
   private Set<String> affectedColumns;
 
 
   public AnnotateStatsProcCtx(ParseContext pctx) {
-    this.setParseContext(pctx);
+    this.pctx = pctx;
     if(pctx != null) {
-      this.setConf(pctx.getConf());
+      this.conf = pctx.getConf();
+      this.uniformWithinRange = HiveConf.getBoolVar(this.conf,
+          HiveConf.ConfVars.HIVE_STATS_RANGE_SELECTIVITY_UNIFORM_DISTRIBUTION);
     } else {
-      this.setConf(null);
+      this.conf = null;
+      this.uniformWithinRange = false;
     }
-    affectedColumns = new HashSet<>();
+    this.andExprStats = null;
+    this.affectedColumns = new HashSet<>();
   }
 
   public HiveConf getConf() {
     return conf;
   }
 
-  public void setConf(HiveConf conf) {
-    this.conf = conf;
+  public boolean isUniformWithinRange() {
+    return uniformWithinRange;
   }
 
   public ParseContext getParseContext() {
     return pctx;
   }
 
-  public void setParseContext(ParseContext pctx) {
-    this.pctx = pctx;
-  }
-
   public Statistics getAndExprStats() {
     return andExprStats;
   }
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 4fc7362..feec457 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.optimizer.stats.annotation;
 
 import java.lang.reflect.Field;
 import java.math.BigDecimal;
+import java.math.RoundingMode;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -911,6 +912,7 @@ public class StatsRulesProcFactory {
       ExprNodeColumnDesc columnDesc;
       ExprNodeConstantDesc constantDesc;
       boolean upperBound;
+      boolean closedBound;
       String boundValue = null;
       if (genFunc.getChildren().get(0) instanceof ExprNodeColumnDesc &&
           genFunc.getChildren().get(1) instanceof ExprNodeConstantDesc) {
@@ -921,14 +923,10 @@ public class StatsRulesProcFactory {
         if (constantDesc.getValue() == null) {
           return 0;
         }
-        if (udf instanceof GenericUDFOPEqualOrGreaterThan ||
-            udf instanceof GenericUDFOPGreaterThan) {
-          boundValue = constantDesc.getValue().toString();
-          upperBound = false;
-        } else {
-          boundValue = constantDesc.getValue().toString();
-          upperBound = true;
-        }
+        boundValue = constantDesc.getValue().toString();
+        upperBound = udf instanceof GenericUDFOPEqualOrLessThan ||
+            udf instanceof GenericUDFOPLessThan;
+        closedBound =  isClosedBound(udf);
       } else if (genFunc.getChildren().get(1) instanceof ExprNodeColumnDesc &&
           genFunc.getChildren().get(0) instanceof ExprNodeConstantDesc) {
         columnDesc = (ExprNodeColumnDesc) genFunc.getChildren().get(1);
@@ -938,14 +936,10 @@ public class StatsRulesProcFactory {
         if (constantDesc.getValue() == null) {
           return 0;
         }
-        if (udf instanceof GenericUDFOPEqualOrGreaterThan ||
-            udf instanceof GenericUDFOPGreaterThan) {
-          boundValue = constantDesc.getValue().toString();
-          upperBound = true;
-        } else {
-          boundValue = constantDesc.getValue().toString();
-          upperBound = false;
-        }
+        boundValue = constantDesc.getValue().toString();
+        upperBound = udf instanceof GenericUDFOPEqualOrGreaterThan ||
+            udf instanceof GenericUDFOPGreaterThan;
+        closedBound = isClosedBound(udf);
       } else {
         // default
         return numRows / 3;
@@ -961,43 +955,63 @@ public class StatsRulesProcFactory {
             byte maxValue = cs.getRange().maxValue.byteValue();
             byte minValue = cs.getRange().minValue.byteValue();
             if (upperBound) {
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && closedBound) {
                 return numRows;
               }
-              if (minValue > value) {
+              if (minValue > value || minValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (value - minValue) / (maxValue - minValue)) * numRows);
+              }
             } else {
-              if (minValue >= value) {
+              if (minValue > value || minValue == value && closedBound) {
                 return numRows;
               }
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (maxValue - value) / (maxValue - minValue)) * numRows);
+              }
             }
           } else if (colTypeLowerCase.equals(serdeConstants.SMALLINT_TYPE_NAME)) {
             short value = Short.parseShort(boundValue);
             short maxValue = cs.getRange().maxValue.shortValue();
             short minValue = cs.getRange().minValue.shortValue();
             if (upperBound) {
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && closedBound) {
                 return numRows;
               }
-              if (minValue > value) {
+              if (minValue > value || minValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (value - minValue) / (maxValue - minValue)) * numRows);
+              }
             } else {
-              if (minValue >= value) {
+              if (minValue > value || minValue == value && closedBound) {
                 return numRows;
               }
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (maxValue - value) / (maxValue - minValue)) * numRows);
+              }
             }
           } else if (colTypeLowerCase.equals(serdeConstants.INT_TYPE_NAME) ||
               colTypeLowerCase.equals(serdeConstants.DATE_TYPE_NAME)) {
             int value;
-            if (colTypeLowerCase == serdeConstants.DATE_TYPE_NAME) {
+            if (colTypeLowerCase.equals(serdeConstants.DATE_TYPE_NAME)) {
               DateWritable writableVal = new DateWritable(java.sql.Date.valueOf(boundValue));
               value = writableVal.getDays();
             } else {
@@ -1007,76 +1021,116 @@ public class StatsRulesProcFactory {
             int maxValue = cs.getRange().maxValue.intValue();
             int minValue = cs.getRange().minValue.intValue();
             if (upperBound) {
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && closedBound) {
                 return numRows;
               }
-              if (minValue > value) {
+              if (minValue > value || minValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (value - minValue) / (maxValue - minValue)) * numRows);
+              }
             } else {
-              if (minValue >= value) {
+              if (minValue > value || minValue == value && closedBound) {
                 return numRows;
               }
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (maxValue - value) / (maxValue - minValue)) * numRows);
+              }
             }
           } else if (colTypeLowerCase.equals(serdeConstants.BIGINT_TYPE_NAME)) {
             long value = Long.parseLong(boundValue);
             long maxValue = cs.getRange().maxValue.longValue();
             long minValue = cs.getRange().minValue.longValue();
             if (upperBound) {
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && closedBound) {
                 return numRows;
               }
-              if (minValue > value) {
+              if (minValue > value || minValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (value - minValue) / (maxValue - minValue)) * numRows);
+              }
             } else {
-              if (minValue >= value) {
+              if (minValue > value || minValue == value && closedBound) {
                 return numRows;
               }
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (maxValue - value) / (maxValue - minValue)) * numRows);
+              }
             }
           } else if (colTypeLowerCase.equals(serdeConstants.FLOAT_TYPE_NAME)) {
             float value = Float.parseFloat(boundValue);
             float maxValue = cs.getRange().maxValue.floatValue();
             float minValue = cs.getRange().minValue.floatValue();
             if (upperBound) {
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && closedBound) {
                 return numRows;
               }
-              if (minValue > value) {
+              if (minValue > value || minValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (value - minValue) / (maxValue - minValue)) * numRows);
+              }
             } else {
-              if (minValue >= value) {
+              if (minValue > value || minValue == value && closedBound) {
                 return numRows;
               }
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (maxValue - value) / (maxValue - minValue)) * numRows);
+              }
             }
           } else if (colTypeLowerCase.equals(serdeConstants.DOUBLE_TYPE_NAME)) {
             double value = Double.parseDouble(boundValue);
             double maxValue = cs.getRange().maxValue.doubleValue();
             double minValue = cs.getRange().minValue.doubleValue();
             if (upperBound) {
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && closedBound) {
                 return numRows;
               }
-              if (minValue > value) {
+              if (minValue > value || minValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (value - minValue) / (maxValue - minValue)) * numRows);
+              }
             } else {
-              if (minValue >= value) {
+              if (minValue > value || minValue == value && closedBound) {
                 return numRows;
               }
-              if (maxValue < value) {
+              if (maxValue < value || maxValue == value && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(((double) (maxValue - value) / (maxValue - minValue)) * numRows);
+              }
             }
           } else if (colTypeLowerCase.startsWith(serdeConstants.DECIMAL_TYPE_NAME)) {
             BigDecimal value = new BigDecimal(boundValue);
@@ -1085,19 +1139,35 @@ public class StatsRulesProcFactory {
             int minComparison = value.compareTo(minValue);
             int maxComparison = value.compareTo(maxValue);
             if (upperBound) {
-              if (maxComparison > 0) {
+              if (maxComparison > 0 || maxComparison == 0 && closedBound) {
                 return numRows;
               }
-              if (minComparison < 0) {
+              if (minComparison < 0 || minComparison == 0 && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(
+                    ((value.subtract(minValue)).divide(maxValue.subtract(minValue), RoundingMode.UP))
+                        .multiply(BigDecimal.valueOf(numRows))
+                        .doubleValue());
+              }
             } else {
-              if (minComparison <= 0) {
+              if (minComparison < 0 || minComparison == 0 && closedBound) {
                 return numRows;
               }
-              if (maxComparison > 0) {
+              if (maxComparison > 0 || maxComparison == 0 && !closedBound) {
                 return 0;
               }
+              if (aspCtx.isUniformWithinRange()) {
+                // Assuming uniform distribution, we can use the range to calculate
+                // new estimate for the number of rows
+                return Math.round(
+                    ((maxValue.subtract(value)).divide(maxValue.subtract(minValue), RoundingMode.UP))
+                        .multiply(BigDecimal.valueOf(numRows))
+                        .doubleValue());
+              }
             }
           }
         } catch (NumberFormatException nfe) {
@@ -1108,9 +1178,15 @@ public class StatsRulesProcFactory {
       return numRows / 3;
     }
 
+    private boolean isClosedBound(GenericUDF udf) {
+      return udf instanceof GenericUDFOPEqualOrGreaterThan ||
+          udf instanceof GenericUDFOPEqualOrLessThan;
+    }
+
     private long evaluateChildExpr(Statistics stats, ExprNodeDesc child,
         AnnotateStatsProcCtx aspCtx, List<String> neededCols,
-        Operator<?> op, long currNumRows) throws SemanticException {
+        Operator<?> op, long currNumRows)
+        throws SemanticException {
 
       long numRows = currNumRows;
 
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 9a00a75..be52709 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
@@ -127,6 +127,10 @@ public class StatsUtils {
 
   private static final Logger LOG = LoggerFactory.getLogger(StatsUtils.class.getName());
 
+  // Range lower limit for date type when not defined (days, heuristic): '1999-01-01'
+  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;
 
   /**
    * Collect table, partition and column level statistics
@@ -944,7 +948,7 @@ public class StatsUtils {
     } else if (colTypeLowerCase.equals(serdeConstants.DATE_TYPE_NAME)) {
       cs.setAvgColLen(JavaDataModel.get().lengthOfDate());
       // epoch, days since epoch
-      cs.setRange(0, 25201);
+      cs.setRange(DATE_RANGE_LOWER_LIMIT, DATE_RANGE_UPPER_LIMIT);
     } else {
       cs.setAvgColLen(getSizeOfComplexTypes(conf, cinfo.getObjectInspector()));
     }
diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q b/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q
index ec32e9c..0fbb385 100644
--- a/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q
+++ b/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q
@@ -1,4 +1,5 @@
 --! qt:dataset:src
+set hive.stats.filter.range.uniform=false;
 set hive.stats.column.autogather=false;
 set hive.strict.checks.bucketing=false;
 
diff --git a/ql/src/test/queries/clientpositive/retry_failure_stat_changes.q b/ql/src/test/queries/clientpositive/retry_failure_stat_changes.q
index 08fd4a2..7882538 100644
--- a/ql/src/test/queries/clientpositive/retry_failure_stat_changes.q
+++ b/ql/src/test/queries/clientpositive/retry_failure_stat_changes.q
@@ -1,4 +1,5 @@
 SET hive.vectorized.execution.enabled=false;
+set hive.stats.filter.range.uniform=false;
 
 create table tx_n2(a int,u int);
 insert into tx_n2 values (1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(10,10);
diff --git a/ql/src/test/queries/clientpositive/semijoin_reddedup.q b/ql/src/test/queries/clientpositive/semijoin_reddedup.q
index b01e58c..8e298ac 100644
--- a/ql/src/test/queries/clientpositive/semijoin_reddedup.q
+++ b/ql/src/test/queries/clientpositive/semijoin_reddedup.q
@@ -20,6 +20,7 @@ set hive.stats.fetch.column.stats=true;
 set hive.tez.bloom.filter.factor=1.0f;
 set hive.auto.convert.join=false;
 set hive.optimize.shared.work=false;
+set hive.stats.filter.range.uniform=false;
 
 
 create database tpch_test;
diff --git a/ql/src/test/results/clientpositive/annotate_stats_filter.q.out b/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
index 957fd12..588b04c 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
@@ -1101,14 +1101,14 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (locid < 3) (type: boolean)
-              Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1143,14 +1143,14 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (locid > 3) (type: boolean)
-              Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1185,14 +1185,14 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (locid <= 3) (type: boolean)
-              Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1227,14 +1227,14 @@ STAGE PLANS:
             Statistics: Num rows: 8 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (locid >= 3) (type: boolean)
-              Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 510 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/annotate_stats_join_pkfk.q.out b/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out
index 709de13..18f2c6a 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out
@@ -601,32 +601,32 @@ STAGE PLANS:
             Statistics: Num rows: 12 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
             Filter Operator
               predicate: ((s_company_id > 0) and s_store_sk is not null) (type: boolean)
-              Statistics: Num rows: 12 Data size: 96 Basic stats: COMPLETE Column stats: PARTIAL
+              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL
               Select Operator
                 expressions: s_store_sk (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: PARTIAL
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: PARTIAL
           TableScan
             alias: ss
             filterExpr: ((ss_quantity > 10) and ss_store_sk is not null) (type: boolean)
             Statistics: Num rows: 1000 Data size: 7676 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((ss_quantity > 10) and ss_store_sk is not null) (type: boolean)
-              Statistics: Num rows: 321 Data size: 2468 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 876 Data size: 6724 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ss_store_sk (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 321 Data size: 1240 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 876 Data size: 3380 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 321 Data size: 1240 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 876 Data size: 3380 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -635,10 +635,10 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0
-          Statistics: Num rows: 321 Data size: 1284 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 90 Data size: 360 Basic stats: COMPLETE Column stats: PARTIAL
           File Output Operator
             compressed: false
-            Statistics: Num rows: 321 Data size: 1284 Basic stats: COMPLETE Column stats: PARTIAL
+            Statistics: Num rows: 90 Data size: 360 Basic stats: COMPLETE Column stats: PARTIAL
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -763,16 +763,16 @@ STAGE PLANS:
             Statistics: Num rows: 1000 Data size: 7676 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((ss_quantity > 10) and ss_store_sk is not null) (type: boolean)
-              Statistics: Num rows: 321 Data size: 2468 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 876 Data size: 6724 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ss_store_sk (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 321 Data size: 1240 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 876 Data size: 3380 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 321 Data size: 1240 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 876 Data size: 3380 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -781,10 +781,10 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0
-          Statistics: Num rows: 321 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 876 Data size: 3504 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 321 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 876 Data size: 3504 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1181,16 +1181,16 @@ STAGE PLANS:
             Statistics: Num rows: 1000 Data size: 7676 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((ss_quantity > 10) and ss_store_sk is not null) (type: boolean)
-              Statistics: Num rows: 321 Data size: 2468 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 876 Data size: 6724 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ss_store_sk (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 321 Data size: 1240 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 876 Data size: 3380 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 321 Data size: 1240 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 876 Data size: 3380 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
             filterExpr: s_store_sk is not null (type: boolean)
@@ -1215,7 +1215,7 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 321 Data size: 2524 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 876 Data size: 6884 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -1231,7 +1231,7 @@ STAGE PLANS:
               key expressions: _col0 (type: int)
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
-              Statistics: Num rows: 321 Data size: 2524 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 876 Data size: 6884 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col1 (type: int)
           TableScan
             alias: s1
@@ -1257,14 +1257,14 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col1
-          Statistics: Num rows: 321 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 876 Data size: 3504 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col1 (type: int)
             outputColumnNames: _col0
-            Statistics: Num rows: 321 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 876 Data size: 3504 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 321 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 876 Data size: 3504 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/cbo_rp_auto_join1.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
index 57a78e0..425b920 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
@@ -664,11 +664,11 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: key
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -765,11 +765,11 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: key
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -888,11 +888,11 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: key
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -1001,11 +1001,11 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 8) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: key
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -1291,11 +1291,11 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: key
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -1384,11 +1384,11 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: key
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -1495,11 +1495,11 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: key
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
diff --git a/ql/src/test/results/clientpositive/correlationoptimizer13.q.out b/ql/src/test/results/clientpositive/correlationoptimizer13.q.out
index 058d4ed..4c000f8 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer13.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer13.q.out
@@ -55,19 +55,19 @@ STAGE PLANS:
             Statistics: Num rows: 1028 Data size: 97660 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((c1 < 120) and c3 is not null) (type: boolean)
-              Statistics: Num rows: 342 Data size: 32490 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 248 Data size: 23560 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: count()
                 keys: c1 (type: int), c3 (type: string)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
-                  Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col2 (type: bigint)
           TableScan
             alias: x1
@@ -75,35 +75,35 @@ STAGE PLANS:
             Statistics: Num rows: 1028 Data size: 101772 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((c2 > 100) and (c1 < 120) and c3 is not null) (type: boolean)
-              Statistics: Num rows: 114 Data size: 11286 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 198 Data size: 19602 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: c1 (type: int), c3 (type: string)
                 outputColumnNames: c1, c3
-                Statistics: Num rows: 114 Data size: 11286 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 198 Data size: 19602 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: c1 (type: int), c3 (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 57 Data size: 5871 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 99 Data size: 10197 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int), _col1 (type: string)
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
-                    Statistics: Num rows: 57 Data size: 5871 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 99 Data size: 10197 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Demux Operator
-          Statistics: Num rows: 228 Data size: 23484 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 223 Data size: 22969 Basic stats: COMPLETE Column stats: COMPLETE
           Group By Operator
             aggregations: count(VALUE._col0)
             keys: KEY._col0 (type: int), KEY._col1 (type: string)
             mode: mergepartial
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
             Mux Operator
-              Statistics: Num rows: 342 Data size: 35226 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 248 Data size: 25544 Basic stats: COMPLETE Column stats: COMPLETE
               Join Operator
                 condition map:
                      Inner Join 0 to 1
@@ -111,11 +111,11 @@ STAGE PLANS:
                   0 _col0 (type: int), _col1 (type: string)
                   1 _col0 (type: int), _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: string), _col2 (type: bigint), _col5 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
                     table:
@@ -127,9 +127,9 @@ STAGE PLANS:
             keys: KEY._col0 (type: int), KEY._col1 (type: string)
             mode: mergepartial
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
             Mux Operator
-              Statistics: Num rows: 342 Data size: 35226 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 248 Data size: 25544 Basic stats: COMPLETE Column stats: COMPLETE
               Join Operator
                 condition map:
                      Inner Join 0 to 1
@@ -137,11 +137,11 @@ STAGE PLANS:
                   0 _col0 (type: int), _col1 (type: string)
                   1 _col0 (type: int), _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: string), _col2 (type: bigint), _col5 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
                     table:
@@ -156,16 +156,16 @@ STAGE PLANS:
             Reduce Output Operator
               key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string), _col4 (type: bigint), _col5 (type: bigint)
               sort order: ++++++
-              Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: bigint), KEY.reducesinkkey5 (type: bigint)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 272 Data size: 28098 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/correlationoptimizer9.q.out b/ql/src/test/results/clientpositive/correlationoptimizer9.q.out
index 506fbe1..73d3450 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer9.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer9.q.out
@@ -56,19 +56,19 @@ STAGE PLANS:
             Statistics: Num rows: 1028 Data size: 4112 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((c1 < 120) and (c1 > 100)) (type: boolean)
-              Statistics: Num rows: 114 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 198 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: count()
                 keys: c1 (type: int)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
       Execution mode: vectorized
       Reduce Operator Tree:
@@ -77,7 +77,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -93,14 +93,14 @@ STAGE PLANS:
               key expressions: _col0 (type: int)
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
-              Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col1 (type: bigint)
           TableScan
             Reduce Output Operator
               key expressions: _col0 (type: int)
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
-              Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Join Operator
@@ -110,14 +110,14 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 34 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 59 Data size: 1416 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: int), _col2 (type: int), _col1 (type: bigint), _col3 (type: bigint)
             outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 34 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 59 Data size: 1416 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 34 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 59 Data size: 1416 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -132,19 +132,19 @@ STAGE PLANS:
             Statistics: Num rows: 1028 Data size: 4112 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((c2 > 100) and (c2 < 120)) (type: boolean)
-              Statistics: Num rows: 114 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 198 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: count()
                 keys: c2 (type: int)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
       Execution mode: vectorized
       Reduce Operator Tree:
@@ -153,7 +153,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -228,19 +228,19 @@ STAGE PLANS:
             Statistics: Num rows: 1028 Data size: 4112 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((c1 < 120) and (c1 > 100)) (type: boolean)
-              Statistics: Num rows: 114 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 198 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: count()
                 keys: c1 (type: int)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
           TableScan
             alias: x1
@@ -248,31 +248,31 @@ STAGE PLANS:
             Statistics: Num rows: 1028 Data size: 4112 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((c2 > 100) and (c2 < 120)) (type: boolean)
-              Statistics: Num rows: 114 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 198 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: count()
                 keys: c2 (type: int)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Demux Operator
-          Statistics: Num rows: 68 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 118 Data size: 1416 Basic stats: COMPLETE Column stats: COMPLETE
           Group By Operator
             aggregations: count(VALUE._col0)
             keys: KEY._col0 (type: int)
             mode: mergepartial
             outputColumnNames: _col0, _col1
-            Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
             Mux Operator
-              Statistics: Num rows: 68 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 118 Data size: 1416 Basic stats: COMPLETE Column stats: COMPLETE
               Join Operator
                 condition map:
                      Inner Join 0 to 1
@@ -280,14 +280,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 74 Data size: 897 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 129 Data size: 1557 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col2 (type: int), _col1 (type: bigint), _col3 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 74 Data size: 897 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 129 Data size: 1557 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 74 Data size: 897 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 129 Data size: 1557 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -297,9 +297,9 @@ STAGE PLANS:
             keys: KEY._col0 (type: int)
             mode: mergepartial
             outputColumnNames: _col0, _col1
-            Statistics: Num rows: 34 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 59 Data size: 708 Basic stats: COMPLETE Column stats: COMPLETE
             Mux Operator
-              Statistics: Num rows: 68 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 118 Data size: 1416 Basic stats: COMPLETE Column stats: COMPLETE
               Join Operator
                 condition map:
                      Inner Join 0 to 1
@@ -307,14 +307,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 74 Data size: 897 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 129 Data size: 1557 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col2 (type: int), _col1 (type: bigint), _col3 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 74 Data size: 897 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 129 Data size: 1557 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 74 Data size: 897 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 129 Data size: 1557 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -389,19 +389,19 @@ STAGE PLANS:
             Statistics: Num rows: 1028 Data size: 97660 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((c1 < 120) and c3 is not null) (type: boolean)
-              Statistics: Num rows: 342 Data size: 32490 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 248 Data size: 23560 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: count()
                 keys: c1 (type: int), c3 (type: string)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
-                  Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col2 (type: bigint)
       Execution mode: vectorized
       Reduce Operator Tree:
@@ -410,7 +410,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -426,14 +426,14 @@ STAGE PLANS:
               key expressions: _col0 (type: int), _col1 (type: string)
               sort order: ++
               Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
-              Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col2 (type: bigint)
           TableScan
             Reduce Output Operator
               key expressions: _col0 (type: int), _col1 (type: string)
               sort order: ++
               Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
-              Statistics: Num rows: 57 Data size: 5871 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 99 Data size: 10197 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Join Operator
@@ -443,14 +443,14 @@ STAGE PLANS:
             0 _col0 (type: int), _col1 (type: string)
             1 _col0 (type: int), _col1 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 94 Data size: 19364 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 163 Data size: 33578 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: string), _col2 (type: bigint), _col5 (type: bigint)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 94 Data size: 19364 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 163 Data size: 33578 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 94 Data size: 19364 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 163 Data size: 33578 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -465,23 +465,23 @@ STAGE PLANS:
             Statistics: Num rows: 1028 Data size: 101772 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((c2 > 100) and (c1 < 120) and c3 is not null) (type: boolean)
-              Statistics: Num rows: 114 Data size: 11286 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 198 Data size: 19602 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: c1 (type: int), c3 (type: string)
                 outputColumnNames: c1, c3
-                Statistics: Num rows: 114 Data size: 11286 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 198 Data size: 19602 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: c1 (type: int), c3 (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 57 Data size: 5871 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 99 Data size: 10197 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int), _col1 (type: string)
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
-                    Statistics: Num rows: 57 Data size: 5871 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 99 Data size: 10197 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
       Execution mode: vectorized
       Reduce Operator Tree:
@@ -490,7 +490,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 57 Data size: 5871 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 99 Data size: 10197 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -565,19 +565,19 @@ STAGE PLANS:
             Statistics: Num rows: 1028 Data size: 97660 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((c1 < 120) and c3 is not null) (type: boolean)
-              Statistics: Num rows: 342 Data size: 32490 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 248 Data size: 23560 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: count()
                 keys: c1 (type: int), c3 (type: string)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
-                  Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col2 (type: bigint)
           TableScan
             alias: x1
@@ -585,35 +585,35 @@ STAGE PLANS:
             Statistics: Num rows: 1028 Data size: 101772 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((c2 > 100) and (c1 < 120) and c3 is not null) (type: boolean)
-              Statistics: Num rows: 114 Data size: 11286 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 198 Data size: 19602 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: c1 (type: int), c3 (type: string)
                 outputColumnNames: c1, c3
-                Statistics: Num rows: 114 Data size: 11286 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 198 Data size: 19602 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: c1 (type: int), c3 (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 57 Data size: 5871 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 99 Data size: 10197 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int), _col1 (type: string)
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
-                    Statistics: Num rows: 57 Data size: 5871 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 99 Data size: 10197 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Demux Operator
-          Statistics: Num rows: 228 Data size: 23484 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 223 Data size: 22969 Basic stats: COMPLETE Column stats: COMPLETE
           Group By Operator
             aggregations: count(VALUE._col0)
             keys: KEY._col0 (type: int), KEY._col1 (type: string)
             mode: mergepartial
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
             Mux Operator
-              Statistics: Num rows: 342 Data size: 35226 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 248 Data size: 25544 Basic stats: COMPLETE Column stats: COMPLETE
               Join Operator
                 condition map:
                      Inner Join 0 to 1
@@ -621,14 +621,14 @@ STAGE PLANS:
                   0 _col0 (type: int), _col1 (type: string)
                   1 _col0 (type: int), _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: string), _col2 (type: bigint), _col5 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -638,9 +638,9 @@ STAGE PLANS:
             keys: KEY._col0 (type: int), KEY._col1 (type: string)
             mode: mergepartial
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 171 Data size: 17613 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 124 Data size: 12772 Basic stats: COMPLETE Column stats: COMPLETE
             Mux Operator
-              Statistics: Num rows: 342 Data size: 35226 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 248 Data size: 25544 Basic stats: COMPLETE Column stats: COMPLETE
               Join Operator
                 condition map:
                      Inner Join 0 to 1
@@ -648,14 +648,14 @@ STAGE PLANS:
                   0 _col0 (type: int), _col1 (type: string)
                   1 _col0 (type: int), _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: string), _col2 (type: bigint), _col5 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 272 Data size: 28098 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 376 Data size: 38748 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 272 Data size: 28098 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/folder_predicate.q.out b/ql/src/test/results/clientpositive/folder_predicate.q.out
index 5fe3341..b4c4724 100644
--- a/ql/src/test/results/clientpositive/folder_predicate.q.out
+++ b/ql/src/test/results/clientpositive/folder_predicate.q.out
@@ -100,14 +100,14 @@ STAGE PLANS:
             Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (value is null or (value < 3)) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: value (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -155,14 +155,14 @@ STAGE PLANS:
             Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (value is null or (value > 3)) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: value (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -210,14 +210,14 @@ STAGE PLANS:
             Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (value is null or (value <= 3)) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: value (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -266,14 +266,14 @@ STAGE PLANS:
             Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (value is null or (value >= 3)) (type: boolean)
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: value (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -376,14 +376,14 @@ STAGE PLANS:
             Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (value is null or (value <= 1) or (value > 3)) (type: boolean)
-              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: value (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4 Data size: 16 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/fp_literal_arithmetic.q.out b/ql/src/test/results/clientpositive/fp_literal_arithmetic.q.out
index bbb0d6d..8334128 100644
--- a/ql/src/test/results/clientpositive/fp_literal_arithmetic.q.out
+++ b/ql/src/test/results/clientpositive/fp_literal_arithmetic.q.out
@@ -24,11 +24,11 @@ STAGE PLANS:
             Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: l_discount BETWEEN 0.05D AND 0.07D (type: boolean)
-              Statistics: Num rows: 11 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 35 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: l_extendedprice (type: double)
                 outputColumnNames: l_extendedprice
-                Statistics: Num rows: 11 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 35 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(l_extendedprice)
                   minReductionHashAggr: 0.99
@@ -97,11 +97,11 @@ STAGE PLANS:
             Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: l_discount BETWEEN 0.05D AND 0.07D (type: boolean)
-              Statistics: Num rows: 11 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 35 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: l_extendedprice (type: double)
                 outputColumnNames: l_extendedprice
-                Statistics: Num rows: 11 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 35 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(l_extendedprice)
                   minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/join_merging.q.out b/ql/src/test/results/clientpositive/join_merging.q.out
index af840a8..6fd4052 100644
--- a/ql/src/test/results/clientpositive/join_merging.q.out
+++ b/ql/src/test/results/clientpositive/join_merging.q.out
@@ -44,16 +44,16 @@ STAGE PLANS:
             Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((p_size > 10) and p_partkey is not null) (type: boolean)
-              Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 21 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: p_partkey (type: int), p_size (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 21 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 21 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: int)
       Reduce Operator Tree:
         Join Operator
@@ -63,11 +63,11 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0, _col1, _col3
-          Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 21 Data size: 252 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col3 (type: int), _col0 (type: int), _col1 (type: int)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 21 Data size: 252 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -95,7 +95,7 @@ STAGE PLANS:
               key expressions: _col1 (type: int)
               sort order: +
               Map-reduce partition columns: _col1 (type: int)
-              Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 21 Data size: 252 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col0 (type: int), _col2 (type: int)
       Reduce Operator Tree:
         Join Operator
@@ -105,14 +105,14 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col1 (type: int)
           outputColumnNames: _col1, _col3
-          Statistics: Num rows: 34 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 29 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col1 (type: int), _col3 (type: int)
             outputColumnNames: _col0, _col1
-            Statistics: Num rows: 34 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 29 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 34 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 29 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -170,16 +170,16 @@ STAGE PLANS:
             Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((p_size > 10) and p_partkey is not null) (type: boolean)
-              Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 21 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: p_partkey (type: int), p_size (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 21 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 21 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: int)
       Reduce Operator Tree:
         Join Operator
@@ -189,14 +189,14 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0, _col1, _col2, _col4
-          Statistics: Num rows: 8 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 21 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
           Filter Operator
             predicate: (_col4 > _col2) (type: boolean)
-            Statistics: Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 7 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: _col4 (type: int), _col0 (type: int), _col1 (type: int)
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 7 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
                 table:
@@ -224,7 +224,7 @@ STAGE PLANS:
               key expressions: _col1 (type: int)
               sort order: +
               Map-reduce partition columns: _col1 (type: int)
-              Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 7 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col0 (type: int), _col2 (type: int)
       Reduce Operator Tree:
         Join Operator
@@ -234,14 +234,14 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col1 (type: int)
           outputColumnNames: _col1, _col3
-          Statistics: Num rows: 28 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 33 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col1 (type: int), _col3 (type: int)
             outputColumnNames: _col0, _col1
-            Statistics: Num rows: 28 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 33 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 28 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 33 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out b/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
index 4225b7e..362f372 100644
--- a/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
@@ -436,15 +436,15 @@ STAGE PLANS:
                   Statistics: Num rows: 2098 Data size: 41920 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t < 100Y) and (b = 4294967363L)) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: t (type: tinyint), si (type: smallint), i (type: int)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int)
                         sort order: +++
-                        Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
         Reducer 2 
@@ -453,10 +453,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: tinyint), KEY.reducesinkkey1 (type: smallint), KEY.reducesinkkey2 (type: int)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 7 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -510,15 +510,15 @@ STAGE PLANS:
                   Statistics: Num rows: 2098 Data size: 41920 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t < 100Y) and (b = 4294967363L)) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), t (type: tinyint), si (type: smallint), i (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 3 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 616 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
                         sort order: +
-                        Statistics: Num rows: 3 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 616 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: tinyint), _col2 (type: smallint), _col3 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
@@ -528,10 +528,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 3 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 616 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 3 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 7 Data size: 616 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -587,16 +587,16 @@ STAGE PLANS:
                   Statistics: Num rows: 2098 Data size: 706986 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((b = 4294967363L) and (t < 100Y)) (type: boolean)
-                    Statistics: Num rows: 2 Data size: 674 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6 Data size: 2022 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), t (type: tinyint), si (type: smallint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary)
                       outputColumnNames: _col0, _col1, _col2, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                      Statistics: Num rows: 2 Data size: 834 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6 Data size: 2502 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
                         sort order: +
                         Map-reduce partition columns: UDFToInteger(_col0) (type: int)
-                        Statistics: Num rows: 2 Data size: 834 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6 Data size: 2502 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: tinyint), _col2 (type: smallint), _col5 (type: float), _col6 (type: double), _col7 (type: boolean), _col8 (type: string), _col9 (type: timestamp), _col10 (type: decimal(4,2)), _col11 (type: binary)
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
@@ -606,10 +606,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), 0 (type: int), 4294967363L (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: boolean), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: decimal(4,2)), VALUE._col9 (type: binary)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 2 Data size: 834 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6 Data size: 2502 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 2 Data size: 834 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 2502 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                       output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/auto_smb_mapjoin_14.q.out b/ql/src/test/results/clientpositive/llap/auto_smb_mapjoin_14.q.out
index 184a21e..650d2cc 100644
--- a/ql/src/test/results/clientpositive/llap/auto_smb_mapjoin_14.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_smb_mapjoin_14.q.out
@@ -588,11 +588,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -600,21 +600,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -714,11 +714,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -726,21 +726,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -862,11 +862,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -874,21 +874,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1000,11 +1000,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 8) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1012,21 +1012,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 8) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.9230769
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1246,11 +1246,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1258,21 +1258,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1365,11 +1365,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1377,11 +1377,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1389,12 +1389,12 @@ STAGE PLANS:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
                         outputColumnNames: _col0
-                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
         Map 5 
             Map Operator Tree:
@@ -1404,16 +1404,16 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1425,10 +1425,10 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
-                  minReductionHashAggr: 0.8333333
+                  minReductionHashAggr: 0.9166667
                   mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1535,11 +1535,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1547,21 +1547,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_10.q.out b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_10.q.out
index 7c95815..629a8f3 100644
--- a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_10.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_10.q.out
@@ -91,16 +91,16 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 14 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
@@ -111,16 +111,16 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 14 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 6 
@@ -131,16 +131,16 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -152,10 +152,10 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 19 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
-                  minReductionHashAggr: 0.8888889
+                  minReductionHashAggr: 0.94736844
                   mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -256,11 +256,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -268,22 +268,22 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       keys: key (type: int)
                       mode: final
                       outputColumnNames: _col0
-                      Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.6666666
+                          minReductionHashAggr: 0.85714287
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_9.q.out b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_9.q.out
index ed16aef..fe21b2d 100644
--- a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_9.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_9.q.out
@@ -714,11 +714,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -726,21 +726,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -840,11 +840,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -852,21 +852,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -988,11 +988,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1000,21 +1000,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1126,11 +1126,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 8) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1138,21 +1138,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 8) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.9230769
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1372,11 +1372,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1384,21 +1384,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1482,11 +1482,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1494,21 +1494,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1603,11 +1603,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1615,11 +1615,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1627,12 +1627,12 @@ STAGE PLANS:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
                         outputColumnNames: _col0
-                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
         Map 5 
             Map Operator Tree:
@@ -1642,16 +1642,16 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1663,10 +1663,10 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
-                  minReductionHashAggr: 0.8333333
+                  minReductionHashAggr: 0.9166667
                   mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1773,11 +1773,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1785,21 +1785,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2537,11 +2537,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -2549,21 +2549,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2663,11 +2663,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -2675,21 +2675,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2811,11 +2811,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -2823,21 +2823,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2949,11 +2949,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 8) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -2961,21 +2961,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 8) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.9230769
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3061,11 +3061,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -3073,21 +3073,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3171,11 +3171,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -3183,21 +3183,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3292,11 +3292,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -3304,11 +3304,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -3316,12 +3316,12 @@ STAGE PLANS:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
                         outputColumnNames: _col0
-                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
         Map 5 
             Map Operator Tree:
@@ -3331,16 +3331,16 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -3352,10 +3352,10 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
-                  minReductionHashAggr: 0.8333333
+                  minReductionHashAggr: 0.9166667
                   mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3462,11 +3462,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -3474,21 +3474,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/colstats_date_min_max.q.out b/ql/src/test/results/clientpositive/llap/colstats_date_min_max.q.out
index 5fa9486..ec1dbc2 100644
--- a/ql/src/test/results/clientpositive/llap/colstats_date_min_max.q.out
+++ b/ql/src/test/results/clientpositive/llap/colstats_date_min_max.q.out
@@ -162,8 +162,8 @@ Stage-0
             PARTITION_ONLY_SHUFFLE [RS_12]
               Group By Operator [GBY_11] (rows=1 width=8)
                 Output:["_col0"],aggregations:["count()"]
-                Select Operator [SEL_10] (rows=3 width=56)
-                  Filter Operator [FIL_9] (rows=3 width=56)
+                Select Operator [SEL_10] (rows=2 width=56)
+                  Filter Operator [FIL_9] (rows=2 width=56)
                     predicate:(d < DATE'2010-10-03')
                     TableScan [TS_0] (rows=10 width=56)
                       default@d1,d1,Tbl:COMPLETE,Col:COMPLETE,Output:["d"]
diff --git a/ql/src/test/results/clientpositive/llap/constraints_optimization.q.out b/ql/src/test/results/clientpositive/llap/constraints_optimization.q.out
index 7e7933c..3c45a0d 100644
--- a/ql/src/test/results/clientpositive/llap/constraints_optimization.q.out
+++ b/ql/src/test/results/clientpositive/llap/constraints_optimization.q.out
@@ -1104,16 +1104,16 @@ STAGE PLANS:
                   Statistics: Num rows: 6 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (value1 > 2.0D) (type: boolean)
-                    Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key1 (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1126,10 +1126,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1195,16 +1195,16 @@ STAGE PLANS:
                   Statistics: Num rows: 6 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (value1 > 2.0D) (type: boolean)
-                    Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key1 (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1217,19 +1217,19 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: int)
                   minReductionHashAggr: 0.5
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: int)
-                    Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -1239,14 +1239,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: bigint)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 16 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/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out
index 05819ef..064837e 100644
--- a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out
@@ -4173,19 +4173,19 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 25160 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t > 27Y) or t is null) (type: boolean)
-                    Statistics: Num rows: 352 Data size: 8448 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 804 Data size: 19288 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 352 Data size: 8448 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 804 Data size: 19288 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), 'foo' (type: string), _col4 (type: tinyint)
                         outputColumnNames: si, i, b, f, ds, t
-                        Statistics: Num rows: 352 Data size: 39072 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 804 Data size: 89236 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: compute_stats(si, 'hll'), compute_stats(i, 'hll'), compute_stats(b, 'hll'), compute_stats(f, 'hll')
                           keys: ds (type: string), t (type: tinyint)
-                          minReductionHashAggr: 0.6363636
+                          minReductionHashAggr: 0.840796
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                           Statistics: Num rows: 129 Data size: 230523 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4199,7 +4199,7 @@ STAGE PLANS:
                         key expressions: _col4 (type: tinyint)
                         sort order: +
                         Map-reduce partition columns: _col4 (type: tinyint)
-                        Statistics: Num rows: 352 Data size: 8448 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 804 Data size: 19288 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
             Execution mode: llap
             LLAP IO: no inputs
@@ -4232,7 +4232,7 @@ STAGE PLANS:
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 352 Data size: 8448 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 804 Data size: 19288 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -4431,14 +4431,14 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 25160 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t > 27Y) or t is null) (type: boolean)
-                    Statistics: Num rows: 352 Data size: 8448 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 804 Data size: 19288 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 352 Data size: 8448 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 804 Data size: 19288 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 352 Data size: 8448 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 804 Data size: 19288 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.TextInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -4447,11 +4447,11 @@ STAGE PLANS:
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), 'foo' (type: string), _col4 (type: tinyint)
                         outputColumnNames: si, i, b, f, ds, t
-                        Statistics: Num rows: 352 Data size: 39072 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 804 Data size: 89236 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: compute_stats(si, 'hll'), compute_stats(i, 'hll'), compute_stats(b, 'hll'), compute_stats(f, 'hll')
                           keys: ds (type: string), t (type: tinyint)
-                          minReductionHashAggr: 0.6363636
+                          minReductionHashAggr: 0.840796
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                           Statistics: Num rows: 129 Data size: 230523 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization2.q.out b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization2.q.out
index c6f3852..f66363a 100644
--- a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization2.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization2.q.out
@@ -1224,18 +1224,18 @@ STAGE PLANS:
                   Statistics: Num rows: 24 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((ss_sold_date_sk >= 2452617) and (ss_sold_date_sk <= 2452638)) (type: boolean)
-                    Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 24 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       keys: ss_sold_date_sk (type: int), ss_net_paid_inc_tax (type: float), ss_net_profit (type: float)
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 4 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 12 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: float), _col2 (type: float)
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 12 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -1245,14 +1245,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int), KEY._col1 (type: float), KEY._col2 (type: float)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 4 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: float), _col2 (type: float), _col0 (type: int)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 4 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 4 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                         output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
@@ -1261,20 +1261,20 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: float), _col1 (type: float), _col2 (type: int)
                     outputColumnNames: ss_net_paid_inc_tax, ss_net_profit, ss_sold_date_sk
-                    Statistics: Num rows: 4 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(ss_net_paid_inc_tax, 'hll'), compute_stats(ss_net_profit, 'hll')
                       keys: ss_sold_date_sk (type: int)
                       mode: complete
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 1 Data size: 852 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 1704 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col1 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: int)
                         outputColumnNames: _col0, _col1, _col2
-                        Statistics: Num rows: 1 Data size: 852 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 1704 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 1 Data size: 852 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 1704 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1493,16 +1493,16 @@ STAGE PLANS:
                   Statistics: Num rows: 24 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((ss_sold_date_sk >= 2452617) and (ss_sold_date_sk <= 2452638)) (type: boolean)
-                    Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 24 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ss_net_paid_inc_tax (type: float), ss_net_profit (type: float), ss_sold_date_sk (type: int)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 24 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col2 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col2 (type: int)
-                        Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 24 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: float), _col1 (type: float), _col2 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -1512,10 +1512,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: float), VALUE._col1 (type: float), VALUE._col2 (type: int)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 24 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 24 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                       output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
@@ -1524,20 +1524,20 @@ STAGE PLANS:
                 Select Operator
                   expressions: _col0 (type: float), _col1 (type: float), _col2 (type: int)
                   outputColumnNames: ss_net_paid_inc_tax, ss_net_profit, ss_sold_date_sk
-                  Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 24 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: compute_stats(ss_net_paid_inc_tax, 'hll'), compute_stats(ss_net_profit, 'hll')
                     keys: ss_sold_date_sk (type: int)
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 1 Data size: 852 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 1704 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: _col1 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col0 (type: int)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 1 Data size: 852 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 1704 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 1 Data size: 852 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 1704 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/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
index 6da07d4..0dc5b51 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -2704,16 +2704,16 @@ Stage-0
     Stage-1
       Reducer 3 llap
       File Output Operator [FS_24]
-        Select Operator [SEL_23] (rows=41 width=223)
+        Select Operator [SEL_23] (rows=33 width=223)
           Output:["_col0","_col1","_col2"]
-          Filter Operator [FIL_22] (rows=41 width=229)
+          Filter Operator [FIL_22] (rows=33 width=227)
             predicate:((_col7 is null or (_col4 = 0L) or _col4 is null) and ((_col5 < _col4) is not true or (_col4 = 0L) or _col4 is null or _col7 is not null or _col0 is null) and (_col0 is not null or (_col4 = 0L) or _col4 is null or _col7 is not null))
-            Merge Join Operator [MERGEJOIN_45] (rows=41 width=229)
+            Merge Join Operator [MERGEJOIN_45] (rows=33 width=227)
               Conds:RS_19._col0, _col1=RS_20._col0, _col2(Left Outer),Output:["_col0","_col1","_col2","_col4","_col5","_col7"]
             <-Reducer 2 [SIMPLE_EDGE] llap
               SHUFFLE [RS_19]
                 PartitionCols:_col0, _col1
-                Merge Join Operator [MERGEJOIN_44] (rows=36 width=227)
+                Merge Join Operator [MERGEJOIN_44] (rows=31 width=226)
                   Conds:RS_16._col1=RS_17._col0(Left Outer),Output:["_col0","_col1","_col2","_col4","_col5"]
                 <-Map 1 [SIMPLE_EDGE] llap
                   SHUFFLE [RS_16]
@@ -2725,33 +2725,33 @@ Stage-0
                 <-Reducer 4 [SIMPLE_EDGE] llap
                   SHUFFLE [RS_17]
                     PartitionCols:_col0
-                    Group By Operator [GBY_7] (rows=2 width=114)
+                    Group By Operator [GBY_7] (rows=1 width=114)
                       Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
                     <-Map 1 [SIMPLE_EDGE] llap
                       SHUFFLE [RS_6]
                         PartitionCols:_col0
-                        Group By Operator [GBY_5] (rows=2 width=114)
+                        Group By Operator [GBY_5] (rows=1 width=114)
                           Output:["_col0","_col1","_col2"],aggregations:["count()","count(p_name)"],keys:p_mfgr
-                          Select Operator [SEL_4] (rows=8 width=223)
+                          Select Operator [SEL_4] (rows=5 width=223)
                             Output:["p_name","p_mfgr"]
-                            Filter Operator [FIL_27] (rows=8 width=223)
+                            Filter Operator [FIL_27] (rows=5 width=223)
                               predicate:((p_size < 10) and p_mfgr is not null)
                                Please refer to the previous TableScan [TS_0]
             <-Reducer 5 [SIMPLE_EDGE] llap
               SHUFFLE [RS_20]
                 PartitionCols:_col0, _col2
-                Select Operator [SEL_15] (rows=4 width=223)
+                Select Operator [SEL_15] (rows=2 width=223)
                   Output:["_col0","_col1","_col2"]
-                  Group By Operator [GBY_14] (rows=4 width=219)
+                  Group By Operator [GBY_14] (rows=2 width=219)
                     Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
                   <-Map 1 [SIMPLE_EDGE] llap
                     SHUFFLE [RS_13]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_12] (rows=4 width=219)
+                      Group By Operator [GBY_12] (rows=2 width=219)
                         Output:["_col0","_col1"],keys:p_name, p_mfgr
-                        Select Operator [SEL_11] (rows=8 width=223)
+                        Select Operator [SEL_11] (rows=5 width=223)
                           Output:["p_name","p_mfgr"]
-                          Filter Operator [FIL_28] (rows=8 width=223)
+                          Filter Operator [FIL_28] (rows=5 width=223)
                             predicate:((p_size < 10) and p_mfgr is not null and p_name is not null)
                              Please refer to the previous TableScan [TS_0]
 
@@ -2813,7 +2813,7 @@ Stage-0
                             PARTITION_ONLY_SHUFFLE [RS_6]
                               Group By Operator [GBY_5] (rows=1 width=16)
                                 Output:["_col0","_col1"],aggregations:["sum(p_size)","count(p_size)"]
-                                Filter Operator [FIL_37] (rows=8 width=4)
+                                Filter Operator [FIL_37] (rows=5 width=4)
                                   predicate:(p_size < 10)
                                   TableScan [TS_2] (rows=26 width=4)
                                     default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_size"]
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 f216d64..e919266 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
@@ -30,27 +30,27 @@ Stage-0
     Stage-1
       Reducer 3 vectorized, llap
       File Output Operator [FS_35]
-        Select Operator [SEL_34] (rows=2048 width=552)
+        Select Operator [SEL_34] (rows=4626 width=552)
           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=2048 width=552)
+            Merge Join Operator [MERGEJOIN_27] (rows=4626 width=552)
               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=1365 width=251)
+                Select Operator [SEL_29] (rows=3078 width=251)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
-                  Filter Operator [FIL_28] (rows=1365 width=251)
+                  Filter Operator [FIL_28] (rows=3078 width=251)
                     predicate:cint BETWEEN 1000000 AND 3000000
                     TableScan [TS_0] (rows=12288 width=251)
                       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=1019 width=251)
+                Select Operator [SEL_32] (rows=2298 width=251)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
-                  Filter Operator [FIL_31] (rows=1019 width=251)
+                  Filter Operator [FIL_31] (rows=2298 width=251)
                     predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                     TableScan [TS_3] (rows=12288 width=251)
                       default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["ctinyint","csmallint","cint","cbigint","cfloat","cdouble","cstring1","cstring2","ctimestamp1","ctimestamp2","cboolean1","cboolean2"]
@@ -119,23 +119,23 @@ Stage-0
           PARTITION_ONLY_SHUFFLE [RS_11]
             Group By Operator [GBY_10] (rows=1 width=8)
               Output:["_col0"],aggregations:["count()"]
-              Merge Join Operator [MERGEJOIN_29] (rows=2048 width=8)
+              Merge Join Operator [MERGEJOIN_29] (rows=4626 width=8)
                 Conds:RS_32._col0=RS_35._col0(Inner)
               <-Map 1 [SIMPLE_EDGE] vectorized, llap
                 SHUFFLE [RS_32]
                   PartitionCols:_col0
-                  Select Operator [SEL_31] (rows=1365 width=2)
+                  Select Operator [SEL_31] (rows=3078 width=2)
                     Output:["_col0"]
-                    Filter Operator [FIL_30] (rows=1365 width=2)
+                    Filter Operator [FIL_30] (rows=3078 width=2)
                       predicate:cint BETWEEN 1000000 AND 3000000
                       TableScan [TS_0] (rows=12288 width=2)
                         default@alltypesorc,a,Tbl:COMPLETE,Col:COMPLETE,Output:["cint"]
               <-Map 4 [SIMPLE_EDGE] vectorized, llap
                 SHUFFLE [RS_35]
                   PartitionCols:_col0
-                  Select Operator [SEL_34] (rows=1019 width=2)
+                  Select Operator [SEL_34] (rows=2298 width=2)
                     Output:["_col0"]
-                    Filter Operator [FIL_33] (rows=1019 width=8)
+                    Filter Operator [FIL_33] (rows=2298 width=8)
                       predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                       TableScan [TS_3] (rows=12288 width=8)
                         default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["cint","cbigint"]
@@ -192,34 +192,34 @@ Stage-0
     Stage-1
       Reducer 4 vectorized, llap
       File Output Operator [FS_41]
-        Select Operator [SEL_40] (rows=1024 width=11)
+        Select Operator [SEL_40] (rows=2313 width=11)
           Output:["_col0","_col1"]
         <-Reducer 3 [SIMPLE_EDGE] vectorized, llap
           SHUFFLE [RS_39]
-            Group By Operator [GBY_38] (rows=1024 width=11)
+            Group By Operator [GBY_38] (rows=2313 width=11)
               Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
             <-Reducer 2 [SIMPLE_EDGE] llap
               SHUFFLE [RS_11]
                 PartitionCols:_col0
-                Group By Operator [GBY_10] (rows=1024 width=11)
+                Group By Operator [GBY_10] (rows=2313 width=11)
                   Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                  Merge Join Operator [MERGEJOIN_31] (rows=2048 width=3)
+                  Merge Join Operator [MERGEJOIN_31] (rows=4626 width=3)
                     Conds:RS_34._col1=RS_37._col0(Inner),Output:["_col0"]
                   <-Map 1 [SIMPLE_EDGE] vectorized, llap
                     SHUFFLE [RS_34]
                       PartitionCols:_col1
-                      Select Operator [SEL_33] (rows=1365 width=5)
+                      Select Operator [SEL_33] (rows=3078 width=5)
                         Output:["_col0","_col1"]
-                        Filter Operator [FIL_32] (rows=1365 width=5)
+                        Filter Operator [FIL_32] (rows=3078 width=5)
                           predicate:cint BETWEEN 1000000 AND 3000000
                           TableScan [TS_0] (rows=12288 width=5)
                             default@alltypesorc,a,Tbl:COMPLETE,Col:COMPLETE,Output:["csmallint","cint"]
                   <-Map 5 [SIMPLE_EDGE] vectorized, llap
                     SHUFFLE [RS_37]
                       PartitionCols:_col0
-                      Select Operator [SEL_36] (rows=1019 width=2)
+                      Select Operator [SEL_36] (rows=2298 width=2)
                         Output:["_col0"]
-                        Filter Operator [FIL_35] (rows=1019 width=8)
+                        Filter Operator [FIL_35] (rows=2298 width=8)
                           predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                           TableScan [TS_3] (rows=12288 width=8)
                             default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["cint","cbigint"]
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_2.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_2.q.out
index 587123c..51b8999 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_2.q.out
@@ -70,22 +70,22 @@ STAGE PLANS:
                   Statistics: Num rows: 5 Data size: 1030 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (c > 20) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 5 Data size: 1030 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: a (type: int), b (type: varchar(256))
                       outputColumnNames: a, b
-                      Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 5 Data size: 1030 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: a (type: int), b (type: varchar(256))
-                        minReductionHashAggr: 0.0
+                        minReductionHashAggr: 0.6
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int), _col1 (type: varchar(256))
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: int), _col1 (type: varchar(256))
-                          Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: may be used (ACID table)
         Reducer 2 
@@ -95,14 +95,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int), KEY._col1 (type: varchar(256))
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: varchar(256))
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 90 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1 Data size: 90 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -166,22 +166,22 @@ STAGE PLANS:
                   Statistics: Num rows: 5 Data size: 1030 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (c > 20) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 5 Data size: 1030 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: a (type: int), b (type: varchar(256))
                       outputColumnNames: a, b
-                      Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 5 Data size: 1030 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: a (type: int), b (type: varchar(256))
-                        minReductionHashAggr: 0.0
+                        minReductionHashAggr: 0.6
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int), _col1 (type: varchar(256))
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: int), _col1 (type: varchar(256))
-                          Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: may be used (ACID table)
         Reducer 2 
@@ -191,14 +191,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int), KEY._col1 (type: varchar(256))
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: varchar(256))
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 90 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1 Data size: 90 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -264,14 +264,14 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 618 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (c > 20) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 618 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: a (type: int), b (type: varchar(256))
                       outputColumnNames: a, b
-                      Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 618 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: a (type: int), b (type: varchar(256))
-                        minReductionHashAggr: 0.0
+                        minReductionHashAggr: 0.6666666
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_1.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_1.q.out
index 4a58aee..7f1ec56 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_1.q.out
@@ -182,16 +182,16 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((empid < 120) and deptno is not null) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empid (type: int), deptno (type: int), name (type: varchar(256)), salary (type: float), commission (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col1 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col1 (type: int)
-                        Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col2 (type: varchar(256)), _col3 (type: float), _col4 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -222,14 +222,14 @@ STAGE PLANS:
                   0 _col1 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7
-                Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 594 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col5 (type: int), _col0 (type: int), _col2 (type: varchar(256)), _col3 (type: float), _col4 (type: int), _col6 (type: varchar(256)), _col7 (type: int)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                  Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 594 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 594 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -465,14 +465,14 @@ STAGE PLANS:
                   Statistics: Num rows: 5 Data size: 530 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (empid > 120) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empid (type: int), deptno (type: int), name (type: varchar(256)), salary (type: float), commission (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 2 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -487,14 +487,14 @@ STAGE PLANS:
                   Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (empid < 150) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empid (type: int), deptno (type: int), name (type: varchar(256)), salary (type: float), commission (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 2 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 848 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/materialized_view_rewrite_no_join_opt_2.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_no_join_opt_2.q.out
index 7d5493e..71d44a4 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_no_join_opt_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_no_join_opt_2.q.out
@@ -190,16 +190,16 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (empid < 120) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empid (type: int), name (type: varchar(256)), salary (type: float), commission (type: int), deptno (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col4 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col4 (type: int)
-                        Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: varchar(256)), _col2 (type: float), _col3 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -246,14 +246,14 @@ STAGE PLANS:
                   0 _col4 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col5, _col6, _col7
-                Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 594 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col5 (type: int), _col0 (type: int), _col1 (type: varchar(256)), _col2 (type: float), _col3 (type: int), _col6 (type: varchar(256)), _col7 (type: int)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                  Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 594 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 594 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -513,14 +513,14 @@ STAGE PLANS:
                   Statistics: Num rows: 5 Data size: 530 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (empid > 120) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empid (type: int), deptno (type: int), name (type: varchar(256)), salary (type: float), commission (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 2 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -535,14 +535,14 @@ STAGE PLANS:
                   Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (empid < 150) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empid (type: int), deptno (type: int), name (type: varchar(256)), salary (type: float), commission (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 2 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 848 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/materialized_view_rewrite_part_1.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_part_1.q.out
index 3c51db6..22e88fd 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_part_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_part_1.q.out
@@ -190,16 +190,16 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (empid < 120) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empid (type: int), name (type: varchar(256)), salary (type: float), commission (type: int), deptno (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col4 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col4 (type: int)
-                        Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: varchar(256)), _col2 (type: float), _col3 (type: int)
             Execution mode: llap
             LLAP IO: all inputs
@@ -246,14 +246,14 @@ STAGE PLANS:
                   0 _col4 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col5, _col6, _col7
-                Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 594 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col5 (type: int), _col0 (type: int), _col1 (type: varchar(256)), _col2 (type: float), _col3 (type: int), _col6 (type: varchar(256)), _col7 (type: int)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                  Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 594 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 594 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -442,14 +442,14 @@ STAGE PLANS:
                   Statistics: Num rows: 5 Data size: 530 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (empid > 120) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empid (type: int), deptno (type: int), name (type: varchar(256)), salary (type: float), commission (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 2 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -464,14 +464,14 @@ STAGE PLANS:
                   Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (empid < 150) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empid (type: int), deptno (type: int), name (type: varchar(256)), salary (type: float), commission (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                      Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 2 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 848 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_llap.q.out b/ql/src/test/results/clientpositive/llap/orc_llap.q.out
index b9c5b06..0ad9682 100644
--- a/ql/src/test/results/clientpositive/llap/orc_llap.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_llap.q.out
@@ -269,11 +269,11 @@ STAGE PLANS:
                   Statistics: Num rows: 122880 Data size: 1467736 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((cint > 10) and cbigint is not null) (type: boolean)
-                    Statistics: Num rows: 30577 Data size: 365240 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 45873 Data size: 547936 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: hash(cint,csmallint,cbigint) (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 30577 Data size: 365240 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 45873 Data size: 547936 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0)
                         minReductionHashAggr: 0.99
@@ -347,11 +347,11 @@ STAGE PLANS:
                   Statistics: Num rows: 122880 Data size: 30929630 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((cint > 10) and cbigint is not null) (type: boolean)
-                    Statistics: Num rows: 30577 Data size: 7696590 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 45873 Data size: 11546630 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: 30577 Data size: 7696590 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 45873 Data size: 11546630 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0)
                         minReductionHashAggr: 0.99
@@ -425,11 +425,11 @@ STAGE PLANS:
                   Statistics: Num rows: 122880 Data size: 9173100 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((cint > 5) and (cint < 10)) (type: boolean)
-                    Statistics: Num rows: 13653 Data size: 1019300 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 30720 Data size: 2293300 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: hash(cstring2) (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 13653 Data size: 1019300 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 30720 Data size: 2293300 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0)
                         minReductionHashAggr: 0.99
@@ -753,11 +753,11 @@ STAGE PLANS:
                   Statistics: Num rows: 245760 Data size: 2935456 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((cint > 10) and cbigint is not null) (type: boolean)
-                    Statistics: Num rows: 61153 Data size: 730452 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 91747 Data size: 1095872 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: hash(cint,csmallint,cbigint) (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 61153 Data size: 730452 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 91747 Data size: 1095872 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0)
                         minReductionHashAggr: 0.99
@@ -831,11 +831,11 @@ STAGE PLANS:
                   Statistics: Num rows: 245760 Data size: 61859030 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((cint > 10) and cbigint is not null) (type: boolean)
-                    Statistics: Num rows: 61153 Data size: 15392750 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 91747 Data size: 23093340 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: 61153 Data size: 15392750 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 91747 Data size: 23093340 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0)
                         minReductionHashAggr: 0.99
@@ -909,11 +909,11 @@ STAGE PLANS:
                   Statistics: Num rows: 245760 Data size: 18346100 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((cint > 5) and (cint < 10)) (type: boolean)
-                    Statistics: Num rows: 27306 Data size: 2038500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 61440 Data size: 4586600 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: hash(cstring2) (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 27306 Data size: 2038500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 61440 Data size: 4586600 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_predicate_pushdown.q.out b/ql/src/test/results/clientpositive/llap/orc_predicate_pushdown.q.out
index b210c09..f8cae8a 100644
--- a/ql/src/test/results/clientpositive/llap/orc_predicate_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_predicate_pushdown.q.out
@@ -338,14 +338,14 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 4188 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t < 0Y) and (UDFToInteger(t) > -2)) (type: boolean)
-                    Statistics: Num rows: 116 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: hash(t) (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 116 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0)
-                        minReductionHashAggr: 0.99
+                        minReductionHashAggr: 0.875
                         mode: hash
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -411,14 +411,14 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 4188 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t < 0Y) and (UDFToInteger(t) > -2)) (type: boolean)
-                    Statistics: Num rows: 116 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: hash(t) (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 116 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col0)
-                        minReductionHashAggr: 0.99
+                        minReductionHashAggr: 0.875
                         mode: hash
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -829,15 +829,15 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 118521 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t > 0Y) and (d >= 10.0D) and (d < 12.0D) and UDFToInteger(si) BETWEEN 300 AND 400 and (s like '%son') and (not (s like '%car%'))) (type: boolean)
-                    Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col3 (type: string)
                         sort order: -
-                        Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
             Execution mode: llap
@@ -848,7 +848,7 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
@@ -912,15 +912,15 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 118521 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t > 0Y) and (d >= 10.0D) and (d < 12.0D) and UDFToInteger(si) BETWEEN 300 AND 400 and (s like '%son') and (not (s like '%car%'))) (type: boolean)
-                    Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col3 (type: string)
                         sort order: -
-                        Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
             Execution mode: llap
@@ -931,7 +931,7 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1061,15 +1061,15 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 118521 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t > 10Y) and (t <> 101Y) and (d >= 10) and (d < 12.0D) and (s like '%son') and (not (s like '%car%')) and (t > 0Y) and si BETWEEN 300 AND 400) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col3 (type: string)
                         sort order: -
-                        Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
             Execution mode: llap
@@ -1080,7 +1080,7 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1165,15 +1165,15 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 118521 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t > 10Y) and (t <> 101Y) and (d >= 10) and (d < 12.0D) and (s like '%son') and (not (s like '%car%')) and (t > 0Y) and si BETWEEN 300 AND 400) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col3 (type: string)
                         sort order: -
-                        Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
             Execution mode: llap
@@ -1184,7 +1184,7 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/parquet_predicate_pushdown.q.out b/ql/src/test/results/clientpositive/llap/parquet_predicate_pushdown.q.out
index eff0fd6..9e4c6f7 100644
--- a/ql/src/test/results/clientpositive/llap/parquet_predicate_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/llap/parquet_predicate_pushdown.q.out
@@ -765,15 +765,15 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 118521 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t > 0Y) and (d >= 10.0D) and (d < 12.0D) and UDFToInteger(si) BETWEEN 300 AND 400 and (s like '%son') and (not (s like '%car%'))) (type: boolean)
-                    Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col3 (type: string)
                         sort order: -
-                        Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
             Execution mode: llap
@@ -784,7 +784,7 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
@@ -848,15 +848,15 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 118521 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t > 0Y) and (d >= 10.0D) and (d < 12.0D) and UDFToInteger(si) BETWEEN 300 AND 400 and (s like '%son') and (not (s like '%car%'))) (type: boolean)
-                    Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col3 (type: string)
                         sort order: -
-                        Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
             Execution mode: llap
@@ -867,7 +867,7 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 5 Data size: 565 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 25 Data size: 2825 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1042,15 +1042,15 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 118521 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t > 10Y) and (t <> 101Y) and (d >= 10) and (d < 12.0D) and (s like '%son') and (not (s like '%car%')) and (t > 0Y) and si BETWEEN 300 AND 400) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col3 (type: string)
                         sort order: -
-                        Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
             Execution mode: llap
@@ -1061,7 +1061,7 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1146,15 +1146,15 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 118521 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((t > 10Y) and (t <> 101Y) and (d >= 10) and (d < 12.0D) and (s like '%son') and (not (s like '%car%')) and (t > 0Y) and si BETWEEN 300 AND 400) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col3 (type: string)
                         sort order: -
-                        Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
             Execution mode: llap
@@ -1165,7 +1165,7 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 44 Data size: 4972 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 339 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1262,15 +1262,15 @@ STAGE PLANS:
                   Statistics: Num rows: 1049 Data size: 16784 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((f < 123.2) and (f > 1.92) and (f >= 9.99) and f BETWEEN 1.92 AND 123.2 and (i < 67627) and (i > 60627) and (i >= 60626) and i BETWEEN 60626 AND 67627 and (b < 4294967861L) and (b > 4294967261L) and (b >= 4294967260L) and b BETWEEN 4294967261L AND 4294967861L) (type: boolean)
-                    Statistics: Num rows: 38 Data size: 608 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 911 Data size: 14576 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: f (type: float), i (type: int), b (type: bigint)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 38 Data size: 608 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 911 Data size: 14576 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: float)
                         sort order: -
-                        Statistics: Num rows: 38 Data size: 608 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 911 Data size: 14576 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col1 (type: int), _col2 (type: bigint)
             Execution mode: llap
@@ -1281,7 +1281,7 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: float), VALUE._col0 (type: int), VALUE._col1 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 38 Data size: 608 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 911 Data size: 14576 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/results_cache_with_masking.q.out b/ql/src/test/results/clientpositive/llap/results_cache_with_masking.q.out
index 6b36584..e2dcb1f 100644
--- a/ql/src/test/results/clientpositive/llap/results_cache_with_masking.q.out
+++ b/ql/src/test/results/clientpositive/llap/results_cache_with_masking.q.out
@@ -40,19 +40,19 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-                    Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count()
                       keys: key (type: int)
-                      minReductionHashAggr: 0.5060241
+                      minReductionHashAggr: 0.6
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 41 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 41 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -64,10 +64,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 41 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 41 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -121,19 +121,19 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-                    Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count()
                       keys: key (type: int)
-                      minReductionHashAggr: 0.5060241
+                      minReductionHashAggr: 0.6
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 41 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 41 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -145,10 +145,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 41 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 41 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 24 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/retry_failure_reorder.q.out b/ql/src/test/results/clientpositive/llap/retry_failure_reorder.q.out
index 5c0f1d3..f404daa 100644
--- a/ql/src/test/results/clientpositive/llap/retry_failure_reorder.q.out
+++ b/ql/src/test/results/clientpositive/llap/retry_failure_reorder.q.out
@@ -144,23 +144,23 @@ Stage-0
             PARTITION_ONLY_SHUFFLE [RS_18]
               Group By Operator [GBY_17] (rows=1 width=8)
                 Output:["_col0"],aggregations:["sum(_col0)"]
-                Select Operator [SEL_15] (rows=114 width=12)
+                Select Operator [SEL_15] (rows=98 width=12)
                   Output:["_col0"]
-                  Merge Join Operator [MERGEJOIN_51] (rows=114 width=12)
+                  Merge Join Operator [MERGEJOIN_51] (rows=98 width=12)
                     Conds:RS_12._col1=RS_60._col0(Inner),Output:["_col2","_col4","_col6"]
                   <-Map 6 [SIMPLE_EDGE] vectorized, llap
                     SHUFFLE [RS_60]
                       PartitionCols:_col0
-                      Select Operator [SEL_59] (rows=16 width=8)
+                      Select Operator [SEL_59] (rows=6 width=8)
                         Output:["_col0","_col1"]
-                        Filter Operator [FIL_58] (rows=16 width=8)
+                        Filter Operator [FIL_58] (rows=6 width=8)
                           predicate:((w > 9) and id_uw is not null)
                           TableScan [TS_6] (rows=50 width=8)
                             default@tw,tw,Tbl:COMPLETE,Col:COMPLETE,Output:["id_uw","w"]
                   <-Reducer 2 [SIMPLE_EDGE] llap
                     SHUFFLE [RS_12]
                       PartitionCols:_col1
-                      Merge Join Operator [MERGEJOIN_50] (rows=50 width=12)
+                      Merge Join Operator [MERGEJOIN_50] (rows=115 width=12)
                         Conds:RS_54._col0=RS_57._col0(Inner),Output:["_col1","_col2","_col4"]
                       <-Map 1 [SIMPLE_EDGE] vectorized, llap
                         SHUFFLE [RS_54]
@@ -174,9 +174,9 @@ Stage-0
                       <-Map 5 [SIMPLE_EDGE] vectorized, llap
                         SHUFFLE [RS_57]
                           PartitionCols:_col0
-                          Select Operator [SEL_56] (rows=10 width=8)
+                          Select Operator [SEL_56] (rows=23 width=8)
                             Output:["_col0","_col1"]
-                            Filter Operator [FIL_55] (rows=10 width=8)
+                            Filter Operator [FIL_55] (rows=23 width=8)
                               predicate:((v > 3) and id_uv is not null)
                               TableScan [TS_3] (rows=30 width=8)
                                 default@tv,tv,Tbl:COMPLETE,Col:COMPLETE,Output:["id_uv","v"]
@@ -285,7 +285,7 @@ Stage-0
                       PartitionCols:_col0
                       Select Operator [SEL_59] (runtime: rows=25 width=8)
                         Output:["_col0","_col1"]
-                        Filter Operator [FIL_58] (rows=10 width=8)
+                        Filter Operator [FIL_58] (rows=23 width=8)
                           predicate:((v > 3) and id_uv is not null)
                           TableScan [TS_6] (runtime: rows=30 width=8)
                             default@tv,tv,Tbl:COMPLETE,Col:COMPLETE,Output:["id_uv","v"]
@@ -308,7 +308,7 @@ Stage-0
                           PartitionCols:_col0
                           Select Operator [SEL_56] (runtime: rows=5 width=8)
                             Output:["_col0","_col1"]
-                            Filter Operator [FIL_55] (rows=16 width=8)
+                            Filter Operator [FIL_55] (rows=6 width=8)
                               predicate:((w > 9) and id_uw is not null)
                               TableScan [TS_3] (runtime: rows=50 width=8)
                                 default@tw,tw,Tbl:COMPLETE,Col:COMPLETE,Output:["id_uw","w"]
@@ -429,23 +429,23 @@ Stage-0
             PARTITION_ONLY_SHUFFLE [RS_18]
               Group By Operator [GBY_17] (rows=1 width=8)
                 Output:["_col0"],aggregations:["sum(_col0)"]
-                Select Operator [SEL_15] (rows=114 width=12)
+                Select Operator [SEL_15] (rows=98 width=12)
                   Output:["_col0"]
-                  Merge Join Operator [MERGEJOIN_51] (rows=114 width=12)
+                  Merge Join Operator [MERGEJOIN_51] (rows=98 width=12)
                     Conds:RS_12._col1=RS_60._col0(Inner),Output:["_col2","_col4","_col6"]
                   <-Map 6 [SIMPLE_EDGE] vectorized, llap
                     SHUFFLE [RS_60]
                       PartitionCols:_col0
-                      Select Operator [SEL_59] (rows=16 width=8)
+                      Select Operator [SEL_59] (rows=6 width=8)
                         Output:["_col0","_col1"]
-                        Filter Operator [FIL_58] (rows=16 width=8)
+                        Filter Operator [FIL_58] (rows=6 width=8)
                           predicate:((w > 9) and id_uw is not null)
                           TableScan [TS_6] (rows=50 width=8)
                             default@tw,tw,Tbl:COMPLETE,Col:COMPLETE,Output:["id_uw","w"]
                   <-Reducer 2 [SIMPLE_EDGE] llap
                     SHUFFLE [RS_12]
                       PartitionCols:_col1
-                      Merge Join Operator [MERGEJOIN_50] (rows=50 width=12)
+                      Merge Join Operator [MERGEJOIN_50] (rows=115 width=12)
                         Conds:RS_54._col0=RS_57._col0(Inner),Output:["_col1","_col2","_col4"]
                       <-Map 1 [SIMPLE_EDGE] vectorized, llap
                         SHUFFLE [RS_54]
@@ -459,9 +459,9 @@ Stage-0
                       <-Map 5 [SIMPLE_EDGE] vectorized, llap
                         SHUFFLE [RS_57]
                           PartitionCols:_col0
-                          Select Operator [SEL_56] (rows=10 width=8)
+                          Select Operator [SEL_56] (rows=23 width=8)
                             Output:["_col0","_col1"]
-                            Filter Operator [FIL_55] (rows=10 width=8)
+                            Filter Operator [FIL_55] (rows=23 width=8)
                               predicate:((v > 3) and id_uv is not null)
                               TableScan [TS_3] (rows=30 width=8)
                                 default@tv,tv,Tbl:COMPLETE,Col:COMPLETE,Output:["id_uv","v"]
diff --git a/ql/src/test/results/clientpositive/llap/retry_failure_stat_changes.q.out b/ql/src/test/results/clientpositive/llap/retry_failure_stat_changes.q.out
index c72a512..8ec6624 100644
--- a/ql/src/test/results/clientpositive/llap/retry_failure_stat_changes.q.out
+++ b/ql/src/test/results/clientpositive/llap/retry_failure_stat_changes.q.out
@@ -237,42 +237,42 @@ POSTHOOK: Input: default@tx_n2
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 2 <- Map 1 (BROADCAST_EDGE)
-Reducer 3 <- Map 2 (CUSTOM_SIMPLE_EDGE)
+Map 1 <- Map 3 (BROADCAST_EDGE)
+Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 3 llap
+      Reducer 2 llap
       File Output Operator [FS_15]
         Select Operator [SEL_14] (rows=1 width=4)
           Output:["_col0"]
           Group By Operator [GBY_13] (rows=1 width=8)
             Output:["_col0"],aggregations:["sum(VALUE._col0)"]
-          <-Map 2 [CUSTOM_SIMPLE_EDGE] llap
+          <-Map 1 [CUSTOM_SIMPLE_EDGE] llap
             PARTITION_ONLY_SHUFFLE [RS_12]
               Group By Operator [GBY_11] (rows=1 width=8)
                 Output:["_col0"],aggregations:["sum(_col0)"]
-                Select Operator [SEL_9] (rows=2 width=8)
+                Select Operator [SEL_9] (rows=1 width=8)
                   Output:["_col0"]
-                  Map Join Operator [MAPJOIN_30] (rows=2 width=8)
-                    Conds:RS_6._col0=SEL_5._col0(Inner),Output:["_col1","_col3"]
-                  <-Map 1 [BROADCAST_EDGE] llap
-                    BROADCAST [RS_6]
+                  Map Join Operator [MAPJOIN_30] (rows=1 width=8)
+                    Conds:SEL_2._col0=RS_7._col0(Inner),Output:["_col1","_col3"]
+                  <-Map 3 [BROADCAST_EDGE] llap
+                    BROADCAST [RS_7]
                       PartitionCols:_col0
-                      Select Operator [SEL_2] (rows=2 width=8)
+                      Select Operator [SEL_5] (rows=1 width=8)
                         Output:["_col0","_col1"]
-                        Filter Operator [FIL_18] (rows=2 width=8)
-                          predicate:((u < 10) and a is not null)
-                          TableScan [TS_0] (rows=8 width=8)
-                            default@tx_n2,tx_n2,Tbl:COMPLETE,Col:COMPLETE,Output:["a","u"]
-                  <-Select Operator [SEL_5] (rows=5 width=8)
+                        Filter Operator [FIL_19] (rows=1 width=8)
+                          predicate:((p > 2) and a is not null)
+                          TableScan [TS_3] (rows=5 width=8)
+                            default@px,px,Tbl:COMPLETE,Col:COMPLETE,Output:["a","p"]
+                  <-Select Operator [SEL_2] (rows=2 width=8)
                       Output:["_col0","_col1"]
-                      Filter Operator [FIL_19] (rows=5 width=8)
-                        predicate:((p > 2) and a is not null)
-                        TableScan [TS_3] (rows=5 width=8)
-                          default@px,px,Tbl:COMPLETE,Col:COMPLETE,Output:["a","p"]
+                      Filter Operator [FIL_18] (rows=2 width=8)
+                        predicate:((u < 10) and a is not null)
+                        TableScan [TS_0] (rows=8 width=8)
+                          default@tx_n2,tx_n2,Tbl:COMPLETE,Col:COMPLETE,Output:["a","u"]
 
 PREHOOK: query: select assert_true_oom(1 > sum(u*p)) from tx_n2 join px on (tx_n2.a=px.a) where u<10 and p>2
 PREHOOK: type: QUERY
@@ -280,17 +280,17 @@ PREHOOK: Input: default@px
 PREHOOK: Input: default@tx_n2
 #### A masked pattern was here ####
 Status: Failed
-Vertex failed, vertexName=Reducer 3, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError: assert_true_oom: assertation failed; Simulated OOM
+Vertex failed, vertexName=Reducer 2, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError: assert_true_oom: assertation failed; Simulated OOM
 #### A masked pattern was here ####
 ], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError: assert_true_oom: assertation failed; Simulated OOM
 #### A masked pattern was here ####
-]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Reducer 3] killed/failed due to:OWN_TASK_FAILURE]
+]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Reducer 2] killed/failed due to:OWN_TASK_FAILURE]
 DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0
-FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Reducer 3, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError: assert_true_oom: assertation failed; Simulated OOM
+FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Reducer 2, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError: assert_true_oom: assertation failed; Simulated OOM
 #### A masked pattern was here ####
 ], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError: assert_true_oom: assertation failed; Simulated OOM
 #### A masked pattern was here ####
-]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Reducer 3] killed/failed due to:OWN_TASK_FAILURE]DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0
+]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Reducer 2] killed/failed due to:OWN_TASK_FAILURE]DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0
 PREHOOK: query: select assert_true_oom(2000 > sum(u*p)) from tx_n2 join px on (tx_n2.a=px.a) where u<10 and p>2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@px
diff --git a/ql/src/test/results/clientpositive/llap/runtime_stats_hs2.q.out b/ql/src/test/results/clientpositive/llap/runtime_stats_hs2.q.out
index df8ef0d..1899eff 100644
--- a/ql/src/test/results/clientpositive/llap/runtime_stats_hs2.q.out
+++ b/ql/src/test/results/clientpositive/llap/runtime_stats_hs2.q.out
@@ -64,25 +64,25 @@ Stage-0
           PARTITION_ONLY_SHUFFLE [RS_12]
             Group By Operator [GBY_11] (rows=1 width=8)
               Output:["_col0"],aggregations:["sum(_col0)"]
-              Select Operator [SEL_9] (rows=1 width=8)
+              Select Operator [SEL_9] (rows=4 width=8)
                 Output:["_col0"]
-                Merge Join Operator [MERGEJOIN_30] (rows=1 width=8)
+                Merge Join Operator [MERGEJOIN_30] (rows=4 width=8)
                   Conds:RS_33._col0=RS_36._col0(Inner),Output:["_col0","_col1"]
                 <-Map 1 [SIMPLE_EDGE] vectorized, llap
                   SHUFFLE [RS_33]
                     PartitionCols:_col0
-                    Select Operator [SEL_32] (rows=1 width=4)
+                    Select Operator [SEL_32] (rows=7 width=4)
                       Output:["_col0"]
-                      Filter Operator [FIL_31] (rows=1 width=4)
+                      Filter Operator [FIL_31] (rows=7 width=4)
                         predicate:((u < 10) and (u > 2))
                         TableScan [TS_0] (rows=8 width=4)
                           default@tx_n3,tx_n3,Tbl:COMPLETE,Col:COMPLETE,Output:["u"]
                 <-Map 4 [SIMPLE_EDGE] vectorized, llap
                   SHUFFLE [RS_36]
                     PartitionCols:_col0
-                    Select Operator [SEL_35] (rows=1 width=4)
+                    Select Operator [SEL_35] (rows=4 width=4)
                       Output:["_col0"]
-                      Filter Operator [FIL_34] (rows=1 width=4)
+                      Filter Operator [FIL_34] (rows=4 width=4)
                         predicate:((p > 2) and (p < 10))
                         TableScan [TS_3] (rows=5 width=4)
                           default@px_n0,px_n0,Tbl:COMPLETE,Col:COMPLETE,Output:["p"]
diff --git a/ql/src/test/results/clientpositive/llap/semijoin.q.out b/ql/src/test/results/clientpositive/llap/semijoin.q.out
index 99ed8de..c33b7bb 100644
--- a/ql/src/test/results/clientpositive/llap/semijoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/semijoin.q.out
@@ -518,22 +518,22 @@ STAGE PLANS:
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 15) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col1
-                      Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col1 (type: int), _col1 (type: int)
-                        minReductionHashAggr: 0.28571427
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col1 (type: int)
-                          Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -546,25 +546,25 @@ STAGE PLANS:
                   0 key (type: int)
                   1 _col1 (type: int)
                 outputColumnNames: _col1
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
-                    Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -743,22 +743,22 @@ STAGE PLANS:
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 5) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.57142854
+                        minReductionHashAggr: 0.5294118
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Map 4 
@@ -788,25 +788,25 @@ STAGE PLANS:
                   0 key (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col1
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
-                    Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -862,14 +862,14 @@ STAGE PLANS:
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((key > 5) and (value <= 'val_20')) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: int), _col1 (type: string)
-                        minReductionHashAggr: 0.0
+                        minReductionHashAggr: 0.5
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
@@ -978,22 +978,22 @@ STAGE PLANS:
                   Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 2) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.6666666
+                        minReductionHashAggr: 0.5555556
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Map 4 
@@ -1023,21 +1023,21 @@ STAGE PLANS:
                   0 key (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
                   sort order: ++
-                  Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 558 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/smb_mapjoin_14.q.out b/ql/src/test/results/clientpositive/llap/smb_mapjoin_14.q.out
index 1aebec3..ae6cd4c 100644
--- a/ql/src/test/results/clientpositive/llap/smb_mapjoin_14.q.out
+++ b/ql/src/test/results/clientpositive/llap/smb_mapjoin_14.q.out
@@ -480,11 +480,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -492,21 +492,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -606,11 +606,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -618,21 +618,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -754,11 +754,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -766,21 +766,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -892,11 +892,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 8) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -904,21 +904,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 8) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.9230769
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1138,11 +1138,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1150,21 +1150,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1246,11 +1246,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1258,21 +1258,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1365,11 +1365,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1377,11 +1377,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1389,12 +1389,12 @@ STAGE PLANS:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
                         outputColumnNames: _col0
-                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
         Map 5 
             Map Operator Tree:
@@ -1404,16 +1404,16 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1425,10 +1425,10 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
-                  minReductionHashAggr: 0.8333333
+                  minReductionHashAggr: 0.9166667
                   mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1535,11 +1535,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1547,21 +1547,21 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                       Merge Join Operator
                         condition map:
                              Inner Join 0 to 1
                         keys:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
-                        Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
-                          minReductionHashAggr: 0.75
+                          minReductionHashAggr: 0.8888889
                           mode: hash
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/subquery_in.q.out b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
index 76d6400..e1fc35f 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
@@ -3630,22 +3630,22 @@ STAGE PLANS:
                   Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((p_size < 10) and p_mfgr is not null and p_name is not null) (type: boolean)
-                    Statistics: Num rows: 8 Data size: 1784 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 5 Data size: 1115 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_mfgr (type: string), p_name (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 8 Data size: 1752 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 5 Data size: 1095 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: string), _col1 (type: string)
-                        minReductionHashAggr: 0.375
+                        minReductionHashAggr: 0.6
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 4 Data size: 876 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 438 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string)
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                          Statistics: Num rows: 4 Data size: 876 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 438 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -3658,15 +3658,15 @@ STAGE PLANS:
                   0 _col1 (type: string), _col0 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 4 Data size: 892 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 446 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int)
                   outputColumnNames: _col1, _col3, _col4
-                  Statistics: Num rows: 4 Data size: 1300 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 650 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col3 (type: string), _col4 (type: int)
                     sort order: ++
-                    Statistics: Num rows: 4 Data size: 1300 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 650 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: string)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -3674,10 +3674,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: int)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 4 Data size: 892 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 446 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 4 Data size: 892 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 446 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/subquery_select.q.out b/ql/src/test/results/clientpositive/llap/subquery_select.q.out
index d48ee3d..2ed6090 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_select.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_select.q.out
@@ -3918,14 +3918,14 @@ STAGE PLANS:
                   Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey BETWEEN 10000 AND 20000 (type: boolean)
-                    Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_size (type: int)
                       outputColumnNames: p_size
-                      Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: max(p_size)
-                        minReductionHashAggr: 0.875
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
index 3c7183e..1654067 100644
--- a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
@@ -359,7 +359,7 @@ STAGE PLANS:
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: smallint)
-                  minReductionHashAggr: 0.49951172
+                  minReductionHashAggr: 0.4997838
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: ###Masked### Data size: ###Masked### Basic stats: COMPLETE Column stats: COMPLETE
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 9303b4d..7b1de57 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
@@ -50,16 +50,16 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 3058 Data size: 769960 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1161780 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1175564 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1175564 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 86954 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 86954 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -255,16 +255,16 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 3058 Data size: 769960 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1161780 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1175564 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1175564 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 86954 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 86954 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -460,16 +460,16 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 3058 Data size: 769960 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1161780 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1175564 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1175564 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 86954 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 86954 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 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_vector_dynpart_hashjoin_1.q.out b/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
index 116bb11..7bfa26f 100644
--- a/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
@@ -359,7 +359,7 @@ STAGE PLANS:
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: smallint)
-                  minReductionHashAggr: 0.49951172
+                  minReductionHashAggr: 0.4997838
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: ###Masked### Data size: ###Masked### 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 9303b4d..7b1de57 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
@@ -50,16 +50,16 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 3058 Data size: 769960 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1161780 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1175564 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1175564 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 86954 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 86954 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -255,16 +255,16 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 3058 Data size: 769960 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1161780 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1175564 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1175564 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 86954 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 86954 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -460,16 +460,16 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((csmallint < 100S) and UDFToInteger(csmallint) is not null) (type: boolean)
-                    Statistics: Num rows: 3058 Data size: 769960 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4615 Data size: 1161780 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4615 Data size: 1175564 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: 3058 Data size: 779096 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4615 Data size: 1175564 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1084 Data size: 86954 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: 1083 Data size: 158946 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1084 Data size: 86954 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 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: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 282560 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1713 Data size: 353010 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 282560 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_between_in.q.out b/ql/src/test/results/clientpositive/llap/vector_between_in.q.out
index 99c7f44..7b1d7a7 100644
--- a/ql/src/test/results/clientpositive/llap/vector_between_in.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_between_in.q.out
@@ -492,7 +492,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColumnBetween(col 3:date, left -2, right 1)
                     predicate: cdate BETWEEN DATE'1969-12-30' AND DATE'1970-01-02' (type: boolean)
-                    Statistics: Num rows: 1365 Data size: 37744 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3087 Data size: 85288 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cdate (type: date)
                       outputColumnNames: _col0
@@ -500,7 +500,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [3]
-                      Statistics: Num rows: 1365 Data size: 37744 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3087 Data size: 85288 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: date)
                         sort order: +
@@ -508,7 +508,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: 1365 Data size: 37744 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3087 Data size: 85288 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -536,13 +536,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0]
-                Statistics: Num rows: 1365 Data size: 37744 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3087 Data size: 85288 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 1365 Data size: 37744 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3087 Data size: 85288 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -592,7 +592,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColumnNotBetween(col 3:date, left -610, right 608)
                     predicate: cdate NOT BETWEEN DATE'1968-05-01' AND DATE'1971-09-01' (type: boolean)
-                    Statistics: Num rows: 10924 Data size: 301616 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 462 Data size: 12824 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cdate (type: date)
                       outputColumnNames: _col0
@@ -600,7 +600,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [3]
-                      Statistics: Num rows: 10924 Data size: 301616 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 462 Data size: 12824 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: date)
                         sort order: +
@@ -608,7 +608,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: 10924 Data size: 301616 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 462 Data size: 12824 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -636,13 +636,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0]
-                Statistics: Num rows: 10924 Data size: 301616 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 462 Data size: 12824 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 10924 Data size: 301616 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 462 Data size: 12824 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -692,7 +692,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterDecimalColumnBetween(col 1:decimal(20,10), left -20, right 45.9918918919)
                     predicate: cdecimal1 BETWEEN -20 AND 45.9918918919 (type: boolean)
-                    Statistics: Num rows: 1365 Data size: 114240 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 21 Data size: 1904 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cdecimal1 (type: decimal(20,10))
                       outputColumnNames: _col0
@@ -700,7 +700,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [1]
-                      Statistics: Num rows: 1365 Data size: 114240 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 21 Data size: 1904 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: decimal(20,10))
                         sort order: +
@@ -708,7 +708,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: 1365 Data size: 114240 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 21 Data size: 1904 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -736,13 +736,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0]
-                Statistics: Num rows: 1365 Data size: 114240 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 21 Data size: 1904 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 1365 Data size: 114240 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 21 Data size: 1904 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -792,13 +792,13 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterDecimalColumnNotBetween(col 1:decimal(20,10), left -2000, right 4390.1351351351)
                     predicate: cdecimal1 NOT BETWEEN -2000 AND 4390.1351351351 (type: boolean)
-                    Statistics: Num rows: 10924 Data size: 913472 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12259 Data size: 1025136 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: []
-                      Statistics: Num rows: 10924 Data size: 913472 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 12259 Data size: 1025136 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         Group By Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
index b4fb00f..23639c0 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
@@ -73,7 +73,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterDecimalColGreaterDecimalScalar(col 1:decimal(20,10), val 0), FilterDecimalColLessDecimalScalar(col 1:decimal(20,10), val 12345.5678), FilterDecimalColGreaterDecimalScalar(col 2:decimal(23,14), val 1000), SelectColumnIsNotNull(col 0:double), FilterDecimalColNotEqualDecimalScalar(col 2:decimal(23,14), val 0))
                     predicate: ((cdecimal1 > 0) and (cdecimal1 < 12345.5678) and (cdecimal2 > 1000) and cdouble is not null and (cdecimal2 <> 0)) (type: boolean)
-                    Statistics: Num rows: 751 Data size: 165540 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: (cdecimal1 + cdecimal2) (type: decimal(25,14)), (cdecimal1 - (2 * cdecimal2)) (type: decimal(26,14)), ((cdecimal1 + 2.34) / cdecimal2) (type: decimal(38,13)), (cdecimal1 * (cdecimal2 / 3.4)) (type: decimal(38,17)), (cdecimal1 % 10) (type: decimal(12,10)), UDFToInteger(cdecimal1) (type: int), UDFToShort(cdecimal2) (type: smallint), UDFToByte(cdecimal2) (type: tinyint), UDFToLong(cdecimal1) (type: bigint), UDFToBoolean(cdecimal1) (type: boolean), UDFToDou [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -82,7 +82,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [4, 6, 8, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20]
                           selectExpressions: DecimalColAddDecimalColumn(col 1:decimal(20,10), col 2:decimal(23,14)) -> 4:decimal(25,14), DecimalColSubtractDecimalColumn(col 1:decimal(20,10), col 5:decimal(25,14))(children: DecimalScalarMultiplyDecimalColumn(val 2, col 2:decimal(23,14)) -> 5:decimal(25,14)) -> 6:decimal(26,14), DecimalColDivideDecimalColumn(col 7:decimal(21,10), col 2:decimal(23,14))(children: DecimalColAddDecimalScalar(col 1:decimal(20,10), val 2.34) -> 7:decimal(21,10)) [...]
-                      Statistics: Num rows: 751 Data size: 165540 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: decimal(25,14)), _col1 (type: decimal(26,14)), _col2 (type: decimal(38,13)), _col3 (type: decimal(38,17)), _col4 (type: decimal(12,10)), _col5 (type: int), _col6 (type: smallint), _col7 (type: tinyint), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: double), _col11 (type: float), _col12 (type: string), _col13 (type: timestamp)
                         sort order: ++++++++++++++
@@ -91,7 +91,7 @@ STAGE PLANS:
                             keyColumns: 4:decimal(25,14), 6:decimal(26,14), 8:decimal(38,13), 10:decimal(38,17), 11:decimal(12,10), 12:int, 13:smallint, 14:tinyint, 15:bigint, 16:boolean, 17:double, 18:float, 19:string, 20:timestamp
                             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: 751 Data size: 165540 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -133,19 +133,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13]
-                Statistics: Num rows: 751 Data size: 165540 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                 Limit
                   Number of rows: 10
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 10 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 10 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -250,7 +250,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterDecimal64ColGreaterDecimal64Scalar(col 1:decimal(10,3)/DECIMAL_64, val 0), FilterDecimalColLessDecimalScalar(col 4:decimal(10,3), val 12345.5678)(children: ConvertDecimal64ToDecimal(col 1:decimal(10,3)/DECIMAL_64) -> 4:decimal(10,3)), FilterDecimal64ColGreaterDecimal64Scalar(col 2:decimal(7,2)/DECIMAL_64, val 100000), SelectColumnIsNotNull(col 0:double), FilterDecimal64ColNotEqualDecimal64Scalar(col 2:decimal( [...]
                     predicate: ((cdecimal1 > 0) and (cdecimal1 < 12345.5678) and (cdecimal2 > 1000) and cdouble is not null and (cdecimal2 <> 0)) (type: boolean)
-                    Statistics: Num rows: 751 Data size: 165540 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: (cdecimal1 + cdecimal2) (type: decimal(11,3)), (cdecimal1 - (2 * cdecimal2)) (type: decimal(11,3)), ((cdecimal1 + 2.34) / cdecimal2) (type: decimal(21,11)), (cdecimal1 * (cdecimal2 / 3.4)) (type: decimal(23,9)), (cdecimal1 % 10) (type: decimal(5,3)), UDFToInteger(cdecimal1) (type: int), UDFToShort(cdecimal2) (type: smallint), UDFToByte(cdecimal2) (type: tinyint), UDFToLong(cdecimal1) (type: bigint), UDFToBoolean(cdecimal1) (type: boolean), UDFToDouble(c [...]
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -259,7 +259,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [7, 11, 14, 18, 20, 22, 24, 26, 28, 30, 32, 34, 36, 38]
                           selectExpressions: DecimalColAddDecimalColumn(col 5:decimal(10,3), col 6:decimal(7,2))(children: ConvertDecimal64ToDecimal(col 1:decimal(10,3)/DECIMAL_64) -> 5:decimal(10,3), ConvertDecimal64ToDecimal(col 2:decimal(7,2)/DECIMAL_64) -> 6:decimal(7,2)) -> 7:decimal(11,3), DecimalColSubtractDecimalColumn(col 8:decimal(10,3), col 10:decimal(9,2))(children: ConvertDecimal64ToDecimal(col 1:decimal(10,3)/DECIMAL_64) -> 8:decimal(10,3), DecimalScalarMultiplyDecimalColum [...]
-                      Statistics: Num rows: 751 Data size: 165540 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: decimal(11,3)), _col1 (type: decimal(11,3)), _col2 (type: decimal(21,11)), _col3 (type: decimal(23,9)), _col4 (type: decimal(5,3)), _col5 (type: int), _col6 (type: smallint), _col7 (type: tinyint), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: double), _col11 (type: float), _col12 (type: string), _col13 (type: timestamp)
                         sort order: ++++++++++++++
@@ -268,7 +268,7 @@ STAGE PLANS:
                             keyColumns: 7:decimal(11,3), 11:decimal(11,3), 14:decimal(21,11), 18:decimal(23,9), 20:decimal(5,3), 22:int, 24:smallint, 26:tinyint, 28:bigint, 30:boolean, 32:double, 34:float, 36:string, 38:timestamp
                             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: 751 Data size: 165540 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -310,19 +310,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13]
-                Statistics: Num rows: 751 Data size: 165540 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                 Limit
                   Number of rows: 10
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 10 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 220 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 10 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 220 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_elt.q.out b/ql/src/test/results/clientpositive/llap/vector_elt.q.out
index 9da99a6..39c1fcc 100644
--- a/ql/src/test/results/clientpositive/llap/vector_elt.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_elt.q.out
@@ -37,7 +37,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:tinyint, val 0)
                     predicate: (ctinyint > 0Y) (type: boolean)
-                    Statistics: Num rows: 4096 Data size: 312018 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6046 Data size: 460522 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ((UDFToInteger(ctinyint) % 2) + 1) (type: int), cstring1 (type: string), cint (type: int), elt(((UDFToInteger(ctinyint) % 2) + 1), cstring1, cint) (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
@@ -46,7 +46,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [14, 6, 2, 18]
                           selectExpressions: LongColAddLongScalar(col 13:int, val 1)(children: LongColModuloLongScalar(col 0:int, val 2)(children: col 0:tinyint) -> 13:int) -> 14:int, VectorElt(columns [16, 6, 17])(children: LongColAddLongScalar(col 15:int, val 1)(children: LongColModuloLongScalar(col 0:int, val 2)(children: col 0:tinyint) -> 15:int) -> 16:int, col 6:string, CastLongToString(col 2:int) -> 17:string) -> 18:string
-                      Statistics: Num rows: 4096 Data size: 1069830 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6046 Data size: 1579114 Basic stats: COMPLETE Column stats: COMPLETE
                       Limit
                         Number of rows: 10
                         Limit Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vector_inner_join.q.out b/ql/src/test/results/clientpositive/llap/vector_inner_join.q.out
index 5db54ea..68a0542 100644
--- a/ql/src/test/results/clientpositive/llap/vector_inner_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_inner_join.q.out
@@ -75,7 +75,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (c > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c (type: int)
                       outputColumnNames: _col0
@@ -83,7 +83,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0]
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -102,7 +102,7 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: int)
                           outputColumnNames: _col0
@@ -110,13 +110,13 @@ STAGE PLANS:
                               className: VectorSelectOperator
                               native: true
                               projectedOutputColumnNums: [0]
-                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
                             File Sink Vectorization:
                                 className: VectorFileSinkOperator
                                 native: false
-                            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -153,7 +153,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (a > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: a (type: int)
                       outputColumnNames: _col0
@@ -161,7 +161,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0]
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
@@ -171,7 +171,7 @@ STAGE PLANS:
                             keyColumns: 0:int
                             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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -250,7 +250,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (c > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c (type: int)
                       outputColumnNames: _col0
@@ -258,7 +258,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0]
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Left Semi Join 0 to 1
@@ -321,7 +321,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (a > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: a (type: int)
                       outputColumnNames: _col0
@@ -329,7 +329,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0]
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -339,7 +339,7 @@ STAGE PLANS:
                             vectorProcessingMode: HASH
                             projectedOutputColumnNums: []
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.0
+                        minReductionHashAggr: 0.5
                         mode: hash
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
@@ -467,7 +467,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (c > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c (type: int)
                       outputColumnNames: _col0
@@ -475,7 +475,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0]
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -495,7 +495,7 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: string), _col2 (type: int)
                           outputColumnNames: _col0, _col1
@@ -503,13 +503,13 @@ STAGE PLANS:
                               className: VectorSelectOperator
                               native: true
                               projectedOutputColumnNums: [3, 0]
-                          Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
                             File Sink Vectorization:
                                 className: VectorFileSinkOperator
                                 native: false
-                            Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -546,7 +546,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 1:int, val 2)
                     predicate: (a > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: v1 (type: string), a (type: int)
                       outputColumnNames: _col0, _col1
@@ -554,7 +554,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col1 (type: int)
                         sort order: +
@@ -565,7 +565,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:string
-                        Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -645,7 +645,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 1:int, val 2)
                     predicate: (a > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: v1 (type: string), a (type: int)
                       outputColumnNames: _col0, _col1
@@ -653,7 +653,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col1 (type: int)
                         sort order: +
@@ -664,7 +664,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:string
-                        Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -698,7 +698,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (c > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c (type: int), v2 (type: string)
                       outputColumnNames: _col0, _col1
@@ -706,7 +706,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -727,13 +727,13 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
                           File Sink Vectorization:
                               className: VectorFileSinkOperator
                               native: false
-                          Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -816,7 +816,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (c > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c (type: int), v2 (type: string), (c * 5) (type: int)
                       outputColumnNames: _col0, _col1, _col2
@@ -825,7 +825,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [0, 1, 3]
                           selectExpressions: LongColMultiplyLongScalar(col 0:int, val 5) -> 3:int
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -846,7 +846,7 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2, _col3, _col5
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col3 (type: string), _col5 (type: int), _col2 (type: int), _col1 (type: string)
                           outputColumnNames: _col0, _col1, _col2, _col3
@@ -854,13 +854,13 @@ STAGE PLANS:
                               className: VectorSelectOperator
                               native: true
                               projectedOutputColumnNums: [4, 5, 3, 1]
-                          Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
                             File Sink Vectorization:
                                 className: VectorFileSinkOperator
                                 native: false
-                            Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -897,7 +897,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 1:int, val 2)
                     predicate: (a > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: v1 (type: string), a (type: int), (a * 2) (type: int)
                       outputColumnNames: _col0, _col1, _col2
@@ -906,7 +906,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [0, 1, 3]
                           selectExpressions: LongColMultiplyLongScalar(col 1:int, val 2) -> 3:int
-                      Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col1 (type: int)
                         sort order: +
@@ -917,7 +917,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:string, 3:int
-                        Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string), _col2 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -997,7 +997,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (c > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c (type: int), v2 (type: string)
                       outputColumnNames: _col0, _col1
@@ -1005,7 +1005,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1026,7 +1026,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col2 (type: string), _col1 (type: string), _col0 (type: int)
                           outputColumnNames: _col0, _col1, _col2
@@ -1034,13 +1034,13 @@ STAGE PLANS:
                               className: VectorSelectOperator
                               native: true
                               projectedOutputColumnNums: [3, 1, 0]
-                          Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
                             File Sink Vectorization:
                                 className: VectorFileSinkOperator
                                 native: false
-                            Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1077,7 +1077,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 1:int, val 2)
                     predicate: (a > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: v1 (type: string), a (type: int)
                       outputColumnNames: _col0, _col1
@@ -1085,7 +1085,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col1 (type: int)
                         sort order: +
@@ -1096,7 +1096,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:string
-                        Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -1176,7 +1176,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (c > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c (type: int), v2 (type: string)
                       outputColumnNames: _col0, _col1
@@ -1184,7 +1184,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1205,7 +1205,7 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2, _col3
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col3 (type: int), _col2 (type: string), _col1 (type: string)
                           outputColumnNames: _col0, _col1, _col2
@@ -1213,13 +1213,13 @@ STAGE PLANS:
                               className: VectorSelectOperator
                               native: true
                               projectedOutputColumnNums: [0, 3, 1]
-                          Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
                             File Sink Vectorization:
                                 className: VectorFileSinkOperator
                                 native: false
-                            Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1256,7 +1256,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 1:int, val 2)
                     predicate: (a > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: v1 (type: string), a (type: int)
                       outputColumnNames: _col0, _col1
@@ -1264,7 +1264,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col1 (type: int)
                         sort order: +
@@ -1275,7 +1275,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:string
-                        Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -1355,7 +1355,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 1:int, val 2)
                     predicate: (a > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: v1 (type: string), a (type: int)
                       outputColumnNames: _col0, _col1
@@ -1363,7 +1363,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col1 (type: int)
                         sort order: +
@@ -1374,7 +1374,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:string
-                        Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -1408,7 +1408,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (c > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c (type: int), v2 (type: string)
                       outputColumnNames: _col0, _col1
@@ -1416,7 +1416,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1437,7 +1437,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col2, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: string), _col3 (type: string), _col2 (type: int)
                           outputColumnNames: _col0, _col1, _col2
@@ -1445,13 +1445,13 @@ STAGE PLANS:
                               className: VectorSelectOperator
                               native: true
                               projectedOutputColumnNums: [3, 1, 0]
-                          Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
                             File Sink Vectorization:
                                 className: VectorFileSinkOperator
                                 native: false
-                            Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1534,7 +1534,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 1:int, val 2)
                     predicate: (a > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: v1 (type: string), a (type: int)
                       outputColumnNames: _col0, _col1
@@ -1542,7 +1542,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col1 (type: int)
                         sort order: +
@@ -1553,7 +1553,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:string
-                        Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -1587,7 +1587,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterLongColGreaterLongScalar(col 0:int, val 2)
                     predicate: (c > 2) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c (type: int), v2 (type: string)
                       outputColumnNames: _col0, _col1
@@ -1595,7 +1595,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 1]
-                      Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1616,7 +1616,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: int), _col0 (type: string), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
@@ -1624,13 +1624,13 @@ STAGE PLANS:
                               className: VectorSelectOperator
                               native: true
                               projectedOutputColumnNums: [0, 3, 1]
-                          Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
                             File Sink Vectorization:
                                 className: VectorFileSinkOperator
                                 native: false
-                            Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 2 Data size: 360 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_leftsemi_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out
index 6d6b0f0..bc53389 100644
--- a/ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out
@@ -522,15 +522,15 @@ STAGE PLANS:
                       outputColumnNames: _col1
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col1 (type: string)
                         outputColumnNames: _col0
-                        Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
-                          Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Map 3 
@@ -541,22 +541,22 @@ STAGE PLANS:
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 15) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col1
-                      Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col1 (type: int), _col1 (type: int)
-                        minReductionHashAggr: 0.28571427
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col1 (type: int)
-                          Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -565,10 +565,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -752,22 +752,22 @@ STAGE PLANS:
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 5) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.57142854
+                        minReductionHashAggr: 0.5294118
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
@@ -788,15 +788,15 @@ STAGE PLANS:
                       outputColumnNames: _col1
                       input vertices:
                         1 Map 1
-                      Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col1 (type: string)
                         outputColumnNames: _col0
-                        Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
-                          Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 3 
@@ -805,10 +805,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -870,14 +870,14 @@ STAGE PLANS:
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((key > 5) and (value <= 'val_20')) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: int), _col1 (type: string)
-                        minReductionHashAggr: 0.0
+                        minReductionHashAggr: 0.5
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
@@ -985,22 +985,22 @@ STAGE PLANS:
                   Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 2) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.6666666
+                        minReductionHashAggr: 0.5555556
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
@@ -1021,11 +1021,11 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       input vertices:
                         1 Map 1
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
                         sort order: ++
-                        Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 3 
@@ -1034,10 +1034,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3630,16 +3630,16 @@ STAGE PLANS:
                       outputColumnNames: _col1
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Select Operator
                         expressions: _col1 (type: string)
                         outputColumnNames: _col0
-                        Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
-                          Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Map 3 
@@ -3650,22 +3650,22 @@ STAGE PLANS:
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key < 15) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col1
-                      Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col1 (type: int), _col1 (type: int)
-                        minReductionHashAggr: 0.28571427
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col1 (type: int)
-                          Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -3674,10 +3674,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3862,22 +3862,22 @@ STAGE PLANS:
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 5) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.57142854
+                        minReductionHashAggr: 0.5294118
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
@@ -3898,16 +3898,16 @@ STAGE PLANS:
                       outputColumnNames: _col1
                       input vertices:
                         1 Map 1
-                      Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Select Operator
                         expressions: _col1 (type: string)
                         outputColumnNames: _col0
-                        Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
-                          Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 3 
@@ -3916,10 +3916,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3981,14 +3981,14 @@ STAGE PLANS:
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((key > 5) and (value <= 'val_20')) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: int), _col1 (type: string)
-                        minReductionHashAggr: 0.0
+                        minReductionHashAggr: 0.5
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4097,22 +4097,22 @@ STAGE PLANS:
                   Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 2) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.6666666
+                        minReductionHashAggr: 0.5555556
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
@@ -4133,12 +4133,12 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       input vertices:
                         1 Map 1
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
                         sort order: ++
-                        Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 3 
@@ -4147,10 +4147,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -9791,7 +9791,7 @@ STAGE PLANS:
                       outputColumnNames: _col1
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Select Operator
                         expressions: _col1 (type: string)
@@ -9799,7 +9799,7 @@ STAGE PLANS:
                         Select Vectorization:
                             className: VectorSelectOperator
                             native: true
-                        Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
@@ -9807,7 +9807,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: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -9832,14 +9832,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: (key < 15) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col1
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -9847,10 +9847,10 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col1 (type: int), _col1 (type: int)
-                        minReductionHashAggr: 0.28571427
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
                           sort order: +
@@ -9859,7 +9859,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               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: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -9886,13 +9886,13 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -10144,14 +10144,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: (key > 5) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -10159,10 +10159,10 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.57142854
+                        minReductionHashAggr: 0.5294118
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
@@ -10171,7 +10171,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               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: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -10212,7 +10212,7 @@ STAGE PLANS:
                       outputColumnNames: _col1
                       input vertices:
                         1 Map 1
-                      Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Select Operator
                         expressions: _col1 (type: string)
@@ -10220,7 +10220,7 @@ STAGE PLANS:
                         Select Vectorization:
                             className: VectorSelectOperator
                             native: true
-                        Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
@@ -10228,7 +10228,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: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -10255,13 +10255,13 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -10328,14 +10328,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: ((key > 5) and (value <= 'val_20')) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -10343,7 +10343,7 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col0 (type: int), _col1 (type: string)
-                        minReductionHashAggr: 0.0
+                        minReductionHashAggr: 0.5
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
@@ -10509,14 +10509,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: (key > 2) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -10524,10 +10524,10 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.6666666
+                        minReductionHashAggr: 0.5555556
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
@@ -10536,7 +10536,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -10577,7 +10577,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       input vertices:
                         1 Map 1
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
@@ -10586,7 +10586,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: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -10613,13 +10613,13 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -14232,14 +14232,14 @@ STAGE PLANS:
                       outputColumnNames: _col1
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col1 (type: string)
                         outputColumnNames: _col0
                         Select Vectorization:
                             className: VectorSelectOperator
                             native: true
-                        Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
@@ -14247,7 +14247,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: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -14272,14 +14272,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: (key < 15) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col1
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -14287,10 +14287,10 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col1 (type: int), _col1 (type: int)
-                        minReductionHashAggr: 0.28571427
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
                           sort order: +
@@ -14299,7 +14299,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               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: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -14326,13 +14326,13 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -14581,14 +14581,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: (key > 5) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -14596,10 +14596,10 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.57142854
+                        minReductionHashAggr: 0.5294118
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
@@ -14608,7 +14608,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               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: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -14647,14 +14647,14 @@ STAGE PLANS:
                       outputColumnNames: _col1
                       input vertices:
                         1 Map 1
-                      Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col1 (type: string)
                         outputColumnNames: _col0
                         Select Vectorization:
                             className: VectorSelectOperator
                             native: true
-                        Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
@@ -14662,7 +14662,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: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -14689,13 +14689,13 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -14762,14 +14762,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: ((key > 5) and (value <= 'val_20')) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -14777,7 +14777,7 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col0 (type: int), _col1 (type: string)
-                        minReductionHashAggr: 0.0
+                        minReductionHashAggr: 0.5
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
@@ -14940,14 +14940,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: (key > 2) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -14955,10 +14955,10 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.6666666
+                        minReductionHashAggr: 0.5555556
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
@@ -14967,7 +14967,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -15006,7 +15006,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       input vertices:
                         1 Map 1
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
                         sort order: ++
@@ -15014,7 +15014,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: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -15041,13 +15041,13 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -18613,7 +18613,7 @@ STAGE PLANS:
                       outputColumnNames: _col1
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Select Operator
                         expressions: _col1 (type: string)
@@ -18621,7 +18621,7 @@ STAGE PLANS:
                         Select Vectorization:
                             className: VectorSelectOperator
                             native: true
-                        Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
@@ -18629,7 +18629,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: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -18654,14 +18654,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: (key < 15) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col1
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -18669,10 +18669,10 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col1 (type: int), _col1 (type: int)
-                        minReductionHashAggr: 0.28571427
+                        minReductionHashAggr: 0.0
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
                           sort order: +
@@ -18681,7 +18681,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               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: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -18708,13 +18708,13 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -18964,14 +18964,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: (key > 5) (type: boolean)
-                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 17 Data size: 68 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -18979,10 +18979,10 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.57142854
+                        minReductionHashAggr: 0.5294118
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
@@ -18991,7 +18991,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               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: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -19030,7 +19030,7 @@ STAGE PLANS:
                       outputColumnNames: _col1
                       input vertices:
                         1 Map 1
-                      Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Select Operator
                         expressions: _col1 (type: string)
@@ -19038,7 +19038,7 @@ STAGE PLANS:
                         Select Vectorization:
                             className: VectorSelectOperator
                             native: true
-                        Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
@@ -19046,7 +19046,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: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -19073,13 +19073,13 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 979 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -19146,14 +19146,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: ((key > 5) and (value <= 'val_20')) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -19161,7 +19161,7 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col0 (type: int), _col1 (type: string)
-                        minReductionHashAggr: 0.0
+                        minReductionHashAggr: 0.5
                         mode: hash
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
@@ -19325,14 +19325,14 @@ STAGE PLANS:
                         className: VectorFilterOperator
                         native: true
                     predicate: (key > 2) (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int)
                       outputColumnNames: _col0
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                      Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         Group By Vectorization:
                             className: VectorGroupByOperator
@@ -19340,10 +19340,10 @@ STAGE PLANS:
                             native: false
                             vectorProcessingMode: HASH
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.6666666
+                        minReductionHashAggr: 0.5555556
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
@@ -19352,7 +19352,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -19391,7 +19391,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       input vertices:
                         1 Map 1
-                      Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
@@ -19400,7 +19400,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: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -19427,13 +19427,13 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 558 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_10.q.out b/ql/src/test/results/clientpositive/llap/vectorization_10.q.out
index 3342549..6787949 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_10.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_10.q.out
@@ -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: 9557 Data size: 1937820 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12288 Data size: 2491562 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: 9557 Data size: 1893568 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 12288 Data size: 2434654 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
                         File Sink Vectorization:
                             className: VectorFileSinkOperator
                             native: false
-                        Statistics: Num rows: 9557 Data size: 1893568 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 12288 Data size: 2434654 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 18ebf12..e1d3032 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_13.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_13.q.out
@@ -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: 5461 Data size: 901772 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1386 Data size: 228984 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: 5461 Data size: 901772 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1386 Data size: 228984 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:
@@ -124,7 +124,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: 2730 Data size: 510974 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 693 Data size: 129752 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: 2730 Data size: 510974 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 693 Data size: 129752 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: 1365 Data size: 255540 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 346 Data size: 64822 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 346 Data size: 113262 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.1
         Reducer 3 
             Execution mode: vectorized, llap
@@ -226,7 +226,7 @@ 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 40
                   Limit Vectorization:
@@ -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: 5461 Data size: 901772 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1386 Data size: 228984 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: 5461 Data size: 901772 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1386 Data size: 228984 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:
@@ -483,7 +483,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: 2730 Data size: 510974 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 693 Data size: 129752 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: 2730 Data size: 510974 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 693 Data size: 129752 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: 1365 Data size: 255540 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 346 Data size: 64822 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 346 Data size: 113262 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.1
         Reducer 3 
             Execution mode: vectorized, llap
@@ -562,7 +562,7 @@ 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 40
                   Limit Vectorization:
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 84c1e35..1f7084a 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_14.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_14.q.out
@@ -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: 606 Data size: 105558 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 758 Data size: 132082 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: 606 Data size: 105558 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 758 Data size: 132082 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:
@@ -124,7 +124,7 @@ STAGE PLANS:
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                        Statistics: Num rows: 303 Data size: 52846 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 379 Data size: 66108 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: 303 Data size: 52846 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 379 Data size: 66108 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: 151 Data size: 26432 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 189 Data size: 33008 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: 151 Data size: 36700 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 189 Data size: 45860 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: 151 Data size: 36700 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 189 Data size: 45860 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: 151 Data size: 36700 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 151 Data size: 36700 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 189 Data size: 45860 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 8dcf6d0..e9e8d1f 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_16.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_16.q.out
@@ -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: 4096 Data size: 769522 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6144 Data size: 1154088 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: 4096 Data size: 769522 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count(_col1), sum(_col3), sum(_col1), min(_col1)
                         Group By Vectorization:
@@ -97,7 +97,7 @@ STAGE PLANS:
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                        Statistics: Num rows: 2048 Data size: 303516 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3072 Data size: 455172 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: 2048 Data size: 303516 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 3072 Data size: 455172 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: 1024 Data size: 151758 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1536 Data size: 227586 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: 1024 Data size: 307406 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 1024 Data size: 307406 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1536 Data size: 461058 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 ff11dfa..1b7dc7e 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_17.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_17.q.out
@@ -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: 4096 Data size: 549274 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6141 Data size: 823456 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: 4096 Data size: 1212930 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6141 Data size: 1818460 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: 4096 Data size: 1212930 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6141 Data size: 1818460 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: 4096 Data size: 1212930 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 4096 Data size: 1212930 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6141 Data size: 1818460 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_7.q.out b/ql/src/test/results/clientpositive/llap/vectorization_7.q.out
index ef03189..51ae4a0 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_7.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_7.q.out
@@ -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: 5461 Data size: 1342196 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 11033 Data size: 2711364 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11033 Data size: 1865892 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -147,7 +147,7 @@ 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 25
                   Limit Vectorization:
@@ -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: 5461 Data size: 1342196 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 11033 Data size: 2711364 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 11033 Data size: 1865892 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -386,7 +386,7 @@ 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 25
                   Limit Vectorization:
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 8dcf6d0..e9e8d1f 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_9.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_9.q.out
@@ -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: 4096 Data size: 769522 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6144 Data size: 1154088 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: 4096 Data size: 769522 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6144 Data size: 1154088 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count(_col1), sum(_col3), sum(_col1), min(_col1)
                         Group By Vectorization:
@@ -97,7 +97,7 @@ STAGE PLANS:
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                        Statistics: Num rows: 2048 Data size: 303516 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3072 Data size: 455172 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: 2048 Data size: 303516 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 3072 Data size: 455172 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: 1024 Data size: 151758 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1536 Data size: 227586 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: 1024 Data size: 307406 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 1024 Data size: 307406 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1536 Data size: 461058 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_div0.q.out b/ql/src/test/results/clientpositive/llap/vectorization_div0.q.out
index 3fe514b..d6e28a1 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_div0.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_div0.q.out
@@ -258,7 +258,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterLongColGreaterLongScalar(col 3:bigint, val 0), FilterLongColLessLongScalar(col 3:bigint, val 100000000))
                     predicate: ((cbigint > 0L) and (cbigint < 100000000L)) (type: boolean)
-                    Statistics: Num rows: 1365 Data size: 16320 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3215 Data size: 38416 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: (cbigint - 988888L) (type: bigint), (cdouble / UDFToDouble((cbigint - 988888L))) (type: double), (1.2 / CAST( (cbigint - 988888L) AS decimal(19,0))) (type: decimal(22,21))
                       outputColumnNames: _col0, _col1, _col2
@@ -267,7 +267,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [13, 16, 19]
                           selectExpressions: LongColSubtractLongScalar(col 3:bigint, val 988888) -> 13:bigint, DoubleColDivideDoubleColumn(col 5:double, col 15:double)(children: CastLongToDouble(col 14:bigint)(children: LongColSubtractLongScalar(col 3:bigint, val 988888) -> 14:bigint) -> 15:double) -> 16:double, DecimalScalarDivideDecimalColumn(val 1.2, col 18:decimal(19,0))(children: CastLongToDecimal(col 17:bigint)(children: LongColSubtractLongScalar(col 3:bigint, val 988888) -> 17:big [...]
-                      Statistics: Num rows: 1365 Data size: 174720 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3215 Data size: 411520 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: bigint), _col1 (type: double), _col2 (type: decimal(22,21))
                         sort order: +++
@@ -275,7 +275,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: 1365 Data size: 174720 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3215 Data size: 411520 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -304,7 +304,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2]
-                Statistics: Num rows: 1365 Data size: 174720 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3215 Data size: 411520 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 100
                   Limit Vectorization:
@@ -482,7 +482,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -500.0), FilterDoubleColLessDoubleScalar(col 5:double, val -199.0))
                     predicate: ((cdouble >= -500.0D) and (cdouble < -199.0D)) (type: boolean)
-                    Statistics: Num rows: 1365 Data size: 16320 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 20 Data size: 256 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: (cdouble + 200.0D) (type: double), (UDFToDouble(cbigint) / (cdouble + 200.0D)) (type: double), ((cdouble + 200.0D) / (cdouble + 200.0D)) (type: double), (UDFToDouble(cbigint) / (cdouble + 200.0D)) (type: double), (3.0D / (cdouble + 200.0D)) (type: double), (1.2D / (cdouble + 200.0D)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
@@ -491,7 +491,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [13, 16, 19, 22, 24, 26]
                           selectExpressions: DoubleColAddDoubleScalar(col 5:double, val 200.0) -> 13:double, DoubleColDivideDoubleColumn(col 14:double, col 15:double)(children: CastLongToDouble(col 3:bigint) -> 14:double, DoubleColAddDoubleScalar(col 5:double, val 200.0) -> 15:double) -> 16:double, DoubleColDivideDoubleColumn(col 17:double, col 18:double)(children: DoubleColAddDoubleScalar(col 5:double, val 200.0) -> 17:double, DoubleColAddDoubleScalar(col 5:double, val 200.0) -> 18:doub [...]
-                      Statistics: Num rows: 1365 Data size: 65520 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 20 Data size: 960 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: double), _col1 (type: double), _col2 (type: double), _col3 (type: double), _col4 (type: double), _col5 (type: double)
                         sort order: ++++++
@@ -499,7 +499,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: 1365 Data size: 65520 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 20 Data size: 960 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -528,19 +528,19 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 1, 4, 5]
-                Statistics: Num rows: 1365 Data size: 65520 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 20 Data size: 960 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 100
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                  Statistics: Num rows: 100 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 20 Data size: 960 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     File Sink Vectorization:
                         className: VectorFileSinkOperator
                         native: false
-                    Statistics: Num rows: 100 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 20 Data size: 960 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -706,7 +706,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterLongColGreaterLongScalar(col 2:int, val 500000000), FilterDoubleColGreaterDoubleScalar(col 5:double, val 1.0E9), FilterLongColEqualLongScalar(col 0:tinyint, val 0))
                     predicate: ((cint > 500000000) or (cdouble > 1.0E9D) or (ctinyint = 0Y)) (type: boolean)
-                    Statistics: Num rows: 4191 Data size: 75120 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3378 Data size: 60552 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int), cbigint (type: bigint), ctinyint (type: tinyint), (UDFToDouble(cint) / UDFToDouble((cint - 528534767))) (type: double), (UDFToDouble(cbigint) / UDFToDouble((cbigint - 1018195815L))) (type: double), (UDFToDouble(ctinyint) / UDFToDouble(ctinyint)) (type: double), (cint % (cint - 528534767)) (type: int), (cbigint % (cbigint - 1018195815L)) (type: bigint), (ctinyint % ctinyint) (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -715,7 +715,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [2, 3, 0, 16, 20, 23, 25, 27, 28]
                           selectExpressions: DoubleColDivideDoubleColumn(col 13:double, col 15:double)(children: CastLongToDouble(col 2:int) -> 13:double, CastLongToDouble(col 14:int)(children: LongColSubtractLongScalar(col 2:int, val 528534767) -> 14:int) -> 15:double) -> 16:double, DoubleColDivideDoubleColumn(col 17:double, col 19:double)(children: CastLongToDouble(col 3:bigint) -> 17:double, CastLongToDouble(col 18:bigint)(children: LongColSubtractLongScalar(col 3:bigint, val 10181958 [...]
-                      Statistics: Num rows: 4191 Data size: 217720 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 3378 Data size: 175488 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: tinyint), _col3 (type: double), _col4 (type: double), _col5 (type: double), _col6 (type: int), _col7 (type: bigint), _col8 (type: tinyint)
                         sort order: +++++++++
@@ -723,7 +723,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: 4191 Data size: 217720 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3378 Data size: 175488 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -752,7 +752,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 4191 Data size: 217720 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3378 Data size: 175488 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 100
                   Limit Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_limit.q.out b/ql/src/test/results/clientpositive/llap/vectorization_limit.q.out
index 4ed1978..f17b61e 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_limit.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_limit.q.out
@@ -33,15 +33,15 @@ STAGE PLANS:
                   Statistics: Num rows: 12288 Data size: 183488 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((cint > 0) and (UDFToDouble(cbigint) < cdouble)) (type: boolean)
-                    Statistics: Num rows: 1365 Data size: 20400 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2048 Data size: 30600 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cbigint (type: bigint), cdouble (type: double)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1365 Data size: 16320 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2048 Data size: 24480 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: bigint), _col1 (type: double)
                         sort order: ++
-                        Statistics: Num rows: 1365 Data size: 16320 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2048 Data size: 24480 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -66,7 +66,7 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: bigint), KEY.reducesinkkey1 (type: double)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1365 Data size: 16320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2048 Data size: 24480 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 7
                   Statistics: Num rows: 7 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
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 2eadb1f..8eb523d 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
@@ -374,7 +374,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprOrExpr(children: FilterExprAndExpr(children: FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -26.28), FilterDoubleColGreaterDoubleColumn(col 13:double, col 5:double)(children: CastLongToDouble(col 1:smallint) -> 13:double)), FilterExprAndExpr(children: FilterLongColLessEqualLongScalar(col 3:bigint, val 197), FilterLongColLessLongColumn(col 2:bigint, col 3:bigint)(children: col 2:int)), FilterExprAndExpr(children: FilterDoub [...]
                     predicate: (((cdouble >= -26.28D) and (UDFToDouble(csmallint) > cdouble)) or ((cbigint <= 197L) and (UDFToLong(cint) < cbigint)) or ((cfloat > 79.553) and (cstring2 like '10%')) or ((UDFToFloat(ctinyint) > cfloat) and cstring1 regexp '.*ss.*')) (type: boolean)
-                    Statistics: Num rows: 6826 Data size: 1131534 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 8186 Data size: 1356970 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int), cbigint (type: bigint), csmallint (type: smallint), cdouble (type: double), ctinyint (type: tinyint), UDFToDouble(cbigint) (type: double), (UDFToDouble(cbigint) * UDFToDouble(cbigint)) (type: double), UDFToDouble(csmallint) (type: double), (UDFToDouble(csmallint) * UDFToDouble(csmallint)) (type: double), UDFToDouble(cint) (type: double), (UDFToDouble(cint) * UDFToDouble(cint)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -383,7 +383,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [2, 3, 1, 5, 0, 15, 18, 19, 22, 23, 26]
                           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, CastLongToDouble(col 1:smallint) -> 19:double, DoubleColMultiplyDoubleColumn(col 20:double, col 21:double)(children: CastLongToDouble(col 1:smallint) -> 20:double, CastLongToDouble(col 1:smallint) -> 21:double) -> 22:double, Cas [...]
-                      Statistics: Num rows: 6826 Data size: 1131534 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 8186 Data size: 1356970 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: max(_col0), sum(_col6), sum(_col5), count(_col1), sum(_col8), sum(_col7), count(_col2), max(_col3), sum(_col4), count(_col4), min(_col0), min(_col3), sum(_col10), sum(_col9), count(_col0)
                         Group By Vectorization:
@@ -2516,7 +2516,7 @@ STAGE PLANS:
                         native: true
                         predicateExpression: FilterExprAndExpr(children: FilterDoubleColGreaterDoubleScalar(col 5:double, val 2563.58), FilterExprOrExpr(children: FilterExprAndExpr(children: FilterLongColGreaterEqualLongColumn(col 3:bigint, col 2:bigint)(children: col 2:int), FilterLongColLessLongColumn(col 1:int, col 2:int)(children: col 1:smallint), FilterDoubleColLessDoubleScalar(col 4:float, val -5638.14990234375)), FilterDecimalColEqualDecimalScalar(col 13:decimal(6,2), val 2563.58) [...]
                     predicate: ((cdouble > 2563.58D) and (((cbigint >= UDFToLong(cint)) and (UDFToInteger(csmallint) < cint) and (cfloat < -5638.15)) or (CAST( ctinyint AS decimal(6,2)) = 2563.58) or ((cdouble <= UDFToDouble(cbigint)) and (CAST( cbigint AS decimal(21,2)) < -5638.15)))) (type: boolean)
-                    Statistics: Num rows: 2503 Data size: 59820 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7494 Data size: 179052 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cdouble (type: double), cfloat (type: float), (cdouble * cdouble) (type: double)
                       outputColumnNames: _col0, _col1, _col2
@@ -2525,7 +2525,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [5, 4, 16]
                           selectExpressions: DoubleColMultiplyDoubleColumn(col 5:double, col 5:double) -> 16:double
-                      Statistics: Num rows: 2503 Data size: 59820 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7494 Data size: 179052 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(_col2), sum(_col0), count(_col0), count(_col1), sum(_col1)
                         Group By Vectorization:
@@ -2537,10 +2537,10 @@ STAGE PLANS:
                             vectorProcessingMode: HASH
                             projectedOutputColumnNums: [0, 1, 2, 3, 4]
                         keys: _col0 (type: double)
-                        minReductionHashAggr: 0.5497403
+                        minReductionHashAggr: 0.55004
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 1127 Data size: 51824 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3372 Data size: 155032 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: double)
                           sort order: +
@@ -2549,7 +2549,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: 1127 Data size: 51824 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 3372 Data size: 155032 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -2584,7 +2584,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1127 Data size: 51824 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3372 Data size: 155032 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: double), ((_col1 - ((_col2 * _col2) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END) (type: double), (2563.58D * ((_col1 - ((_col2 * _col2) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END)) (type: double), (- ((_col1 - ((_col2 * _col2) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END)) (type: double), _col4 (type: bigint), ((2563.58D * ((_col1 - ((_col2 * _col2) / _col3)) /  [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -2593,7 +2593,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 12, 20, 28, 4, 37, 55, 5, 59, 68, 73, 81, 82, 2, 84]
                       selectExpressions: DoubleColDivideLongColumn(col 8:double, col 11:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 7:double)(children: DoubleColDivideLongColumn(col 6:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, col 2:double) -> 6:double) -> 7:double) -> 8:double, IfExprNullCondExpr(col 9:boolean, null, col 10:bigint)(children: LongColEqualLongScalar(col 3:bigint, val 1) -> 9:boolean, LongColSubtractLongScalar(col 3:bi [...]
-                  Statistics: Num rows: 1127 Data size: 141984 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3372 Data size: 424792 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: double)
                     sort order: +
@@ -2601,7 +2601,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: 1127 Data size: 141984 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3372 Data size: 424792 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: double), _col4 (type: bigint), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: double), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: double), _col13 (type: double), _col14 (type: double)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -2619,13 +2619,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 13]
-                Statistics: Num rows: 1127 Data size: 141984 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3372 Data size: 424792 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 1127 Data size: 141984 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3372 Data size: 424792 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -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: 5857 Data size: 1240180 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6237 Data size: 1320590 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: 5857 Data size: 1240180 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6237 Data size: 1320590 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:
@@ -3275,7 +3275,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, _col17, _col18
-                        Statistics: Num rows: 2 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 3 Data size: 432 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: boolean)
                           sort order: +
@@ -3284,7 +3284,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               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: 2 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 3 Data size: 432 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col1 (type: float), _col2 (type: bigint), _col3 (type: double), _col4 (type: double), _col5 (type: bigint), _col6 (type: double), _col7 (type: bigint), _col8 (type: bigint), _col9 (type: double), _col10 (type: double), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: double), _col14 (type: double), _col15 (type: bigint), _col16 (type: double), _col17 (type: double), _col18 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -3319,7 +3319,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: boolean)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
-                Statistics: Num rows: 2 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 432 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: boolean), _col1 (type: float), (- _col1) (type: float), (-26.28D / UDFToDouble(_col1)) (type: double), _col2 (type: bigint), (CAST( _col2 AS decimal(19,0)) - 10.175) (type: decimal(23,3)), ((_col3 - ((_col4 * _col4) / _col5)) / CASE WHEN ((_col5 = 1L)) THEN (null) ELSE ((_col5 - 1)) END) (type: double), (((_col3 - ((_col4 * _col4) / _col5)) / CASE WHEN ((_col5 = 1L)) THEN (null) ELSE ((_col5 - 1)) END) % UDFToDouble(_col1)) (type: double), (10. [...]
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25
@@ -3328,7 +3328,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 1, 19, 20, 2, 22, 29, 37, 39, 40, 51, 8, 55, 58, 63, 76, 12, 84, 86, 90, 95, 100, 102, 105, 109]
                       selectExpressions: DoubleColUnaryMinus(col 1:float) -> 19:float, DoubleScalarDivideDoubleColumn(val -26.28, col 1:double)(children: col 1:float) -> 20:double, DecimalColSubtractDecimalScalar(col 21:decimal(19,0), val 10.175)(children: CastLongToDecimal(col 2:bigint) -> 21:decimal(19,0)) -> 22:decimal(23,3), DoubleColDivideLongColumn(col 25:double, col 28:bigint)(children: DoubleColSubtractDoubleColumn(col 3:double, col 24:double)(children: DoubleColDivideLongColumn( [...]
-                  Statistics: Num rows: 2 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: boolean)
                     sort order: +
@@ -3336,7 +3336,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: 2 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: float), _col2 (type: float), _col3 (type: double), _col4 (type: bigint), _col5 (type: decimal(23,3)), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: bigint), _col12 (type: double), _col13 (type: float), _col14 (type: double), _col15 (type: double), _col17 (type: bigint), _col18 (type: double), _col19 (type: decimal(24,3)), _col20 (type: decimal(25,3)), _col21 ( [...]
         Reducer 3 
             Execution mode: vectorized, llap
@@ -3354,13 +3354,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 13, 16, 17, 18, 19, 20, 21, 22, 23, 24]
-                Statistics: Num rows: 2 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 2 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 1800 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/masking_1.q.out b/ql/src/test/results/clientpositive/masking_1.q.out
index 0958385..fd025a4 100644
--- a/ql/src/test/results/clientpositive/masking_1.q.out
+++ b/ql/src/test/results/clientpositive/masking_1.q.out
@@ -32,14 +32,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -88,14 +88,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (key > 0) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -141,14 +141,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (key > 0) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -194,14 +194,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (key > 0) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: reverse(value) (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 83 Data size: 15272 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15272 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -257,16 +257,16 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0) and UDFToDouble(key) is not null) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string), UDFToDouble(key) (type: double)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 83 Data size: 16268 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col2 (type: double)
                   sort order: +
                   Map-reduce partition columns: _col2 (type: double)
-                  Statistics: Num rows: 83 Data size: 16268 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: int), _col1 (type: string)
           TableScan
             alias: srcpart
@@ -293,14 +293,14 @@ STAGE PLANS:
             0 _col2 (type: double)
             1 _col4 (type: double)
           outputColumnNames: _col0, _col1, _col3, _col4, _col5, _col6
-          Statistics: Num rows: 525 Data size: 385350 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 31 Data size: 22754 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 525 Data size: 385350 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 31 Data size: 22754 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 525 Data size: 385350 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 31 Data size: 22754 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -400,14 +400,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (key > 0) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -453,14 +453,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (key > 0) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 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/masking_12.q.out b/ql/src/test/results/clientpositive/masking_12.q.out
index 6140a98..aa0ed40 100644
--- a/ql/src/test/results/clientpositive/masking_12.q.out
+++ b/ql/src/test/results/clientpositive/masking_12.q.out
@@ -50,14 +50,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -360,14 +360,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (key > 6) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 27 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: UDFToInteger((UDFToDouble(key) / 2.0D)) (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 27 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 27 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -423,16 +423,16 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string)
           TableScan
             alias: masking_test_n5
@@ -442,16 +442,16 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (key > 6) and ((key % 2) = 0) and ((UDFToInteger((UDFToDouble(key) / 2.0D)) % 2) = 0) and (UDFToInteger((UDFToDouble(key) / 2.0D)) < 10)) (type: boolean)
-              Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: UDFToInteger((UDFToDouble(key) / 2.0D)) (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -460,14 +460,14 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col1
-          Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col1 (type: string)
             outputColumnNames: _col0
-            Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 184 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/masking_13.q.out b/ql/src/test/results/clientpositive/masking_13.q.out
index 4ca113d..672d65f 100644
--- a/ql/src/test/results/clientpositive/masking_13.q.out
+++ b/ql/src/test/results/clientpositive/masking_13.q.out
@@ -32,14 +32,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -133,14 +133,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (key > 6) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 27 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: UDFToInteger((UDFToDouble(key) / 2.0D)) (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 27 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 27 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 20 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/masking_1_newdb.q.out b/ql/src/test/results/clientpositive/masking_1_newdb.q.out
index a9c8f9d..293da26 100644
--- a/ql/src/test/results/clientpositive/masking_1_newdb.q.out
+++ b/ql/src/test/results/clientpositive/masking_1_newdb.q.out
@@ -50,14 +50,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -106,14 +106,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (key > 0) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 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/masking_5.q.out b/ql/src/test/results/clientpositive/masking_5.q.out
index c833437..57fab2f 100644
--- a/ql/src/test/results/clientpositive/masking_5.q.out
+++ b/ql/src/test/results/clientpositive/masking_5.q.out
@@ -33,14 +33,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key % 2) = 0) and (key < 10)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -85,14 +85,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key % 2) = 0) and (key < 10)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -171,14 +171,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((((hash(key) & 2147483647) % 2) = 0) and ((key % 2) = 0) and (key < 10)) (type: boolean)
-              Statistics: Num rows: 41 Data size: 3895 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 41 Data size: 7708 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3 Data size: 564 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 41 Data size: 7708 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3 Data size: 564 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/masking_8.q.out b/ql/src/test/results/clientpositive/masking_8.q.out
index 939d965..ee11d37 100644
--- a/ql/src/test/results/clientpositive/masking_8.q.out
+++ b/ql/src/test/results/clientpositive/masking_8.q.out
@@ -37,14 +37,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 90500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 15023 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 905 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string), _c2 (type: string), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 83 Data size: 29050 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 1750 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 29050 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 1750 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -93,14 +93,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 90500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 15023 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 905 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string), _c2 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 83 Data size: 22742 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 1370 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 22742 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 1370 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -149,14 +149,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 90500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 15023 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 905 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: INPUT__FILE__NAME (type: string), key (type: int), reverse(value) (type: string), _c2 (type: string), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 83 Data size: 44322 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 2670 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 44322 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 2670 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -231,14 +231,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 433000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 71878 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 4330 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: int), _c1 (type: string), _c2 (type: string), _c3 (type: string), _c4 (type: string), _c5 (type: string), _c6 (type: string), _c7 (type: string), _c8 (type: string), _c9 (type: string), _c10 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 83 Data size: 78186 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 4710 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 78186 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 4710 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/masking_disablecbo_1.q.out b/ql/src/test/results/clientpositive/masking_disablecbo_1.q.out
index 777352f..6ef55df 100644
--- a/ql/src/test/results/clientpositive/masking_disablecbo_1.q.out
+++ b/ql/src/test/results/clientpositive/masking_disablecbo_1.q.out
@@ -32,14 +32,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key % 2) = 0) and (key < 10)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -88,14 +88,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key % 2) = 0) and (key < 10) and (key > 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -141,14 +141,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key % 2) = 0) and (key < 10) and (key > 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -194,14 +194,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key % 2) = 0) and (key < 10) and (key > 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: reverse(value) (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 83 Data size: 15272 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15272 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -257,16 +257,16 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key % 2) = 0) and (key < 10) and UDFToDouble(key) is not null) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 37848 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 2280 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: UDFToDouble(_col0) (type: double)
                   sort order: +
                   Map-reduce partition columns: UDFToDouble(_col0) (type: double)
-                  Statistics: Num rows: 83 Data size: 37848 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 2280 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: int), _col1 (type: string)
           TableScan
             alias: srcpart
@@ -289,14 +289,14 @@ STAGE PLANS:
             0 UDFToDouble(_col0) (type: double)
             1 UDFToDouble(key) (type: double)
           outputColumnNames: _col0, _col1, _col5, _col6, _col7, _col8
-          Statistics: Num rows: 525 Data size: 385350 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 31 Data size: 22754 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: int), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 525 Data size: 385350 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 31 Data size: 22754 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 525 Data size: 385350 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 31 Data size: 22754 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -396,14 +396,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key % 2) = 0) and (key < 10) and (key > 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -449,14 +449,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key % 2) = 0) and (key < 10) and (key > 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), reverse(value) (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 15604 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 940 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/masking_mv.q.out b/ql/src/test/results/clientpositive/masking_mv.q.out
index a1f5582..e685352 100644
--- a/ql/src/test/results/clientpositive/masking_mv.q.out
+++ b/ql/src/test/results/clientpositive/masking_mv.q.out
@@ -217,14 +217,14 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (0 = (key % 2))) (type: boolean)
-              Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -291,28 +291,28 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (0 = (key % 2))) (type: boolean)
-              Statistics: Num rows: 83 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: key (type: int)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 41 Data size: 164 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 41 Data size: 164 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 41 Data size: 164 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 41 Data size: 164 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -364,32 +364,32 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: reverse(value) (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   keys: _col0 (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
-                  Statistics: Num rows: 41 Data size: 7544 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 41 Data size: 7544 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 41 Data size: 7544 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 41 Data size: 7544 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -441,23 +441,23 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and ((key % 2) = 0)) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: reverse(value) (type: string), key (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col1)
                   keys: _col0 (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 41 Data size: 7872 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 41 Data size: 7872 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
       Execution mode: vectorized
       Reduce Operator Tree:
@@ -466,10 +466,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 41 Data size: 7872 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 41 Data size: 7872 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -535,32 +535,32 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (0 = (key % 2))) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: reverse(value) (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   keys: _col0 (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
-                  Statistics: Num rows: 41 Data size: 7544 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 41 Data size: 7544 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 41 Data size: 7544 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 41 Data size: 7544 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -616,23 +616,23 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((key < 10) and (0 = (key % 2))) (type: boolean)
-              Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: reverse(value) (type: string), key (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5 Data size: 475 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col1)
                   keys: _col0 (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 41 Data size: 7872 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 41 Data size: 7872 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
       Execution mode: vectorized
       Reduce Operator Tree:
@@ -641,10 +641,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 41 Data size: 7872 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 41 Data size: 7872 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 384 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/nullability_transitive_inference.q.out b/ql/src/test/results/clientpositive/nullability_transitive_inference.q.out
index 879eaa2..c14e3cc 100644
--- a/ql/src/test/results/clientpositive/nullability_transitive_inference.q.out
+++ b/ql/src/test/results/clientpositive/nullability_transitive_inference.q.out
@@ -139,16 +139,16 @@ STAGE PLANS:
             Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (deptno >= 20) (type: boolean)
-              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: deptno (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Join Operator
           condition map:
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 0cbd846..0cc7e67 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_10.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_10.q.out
@@ -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: 9557 Data size: 1937820 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 12288 Data size: 2491562 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: 9557 Data size: 1893568 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12288 Data size: 2434654 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 9557 Data size: 1893568 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12288 Data size: 2434654 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 1198427..134ce1f 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_13.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_13.q.out
@@ -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: 5461 Data size: 901772 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1386 Data size: 228984 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: 5461 Data size: 901772 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1386 Data size: 228984 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: 2730 Data size: 510974 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 693 Data size: 129752 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: 2730 Data size: 510974 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 693 Data size: 129752 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: 1365 Data size: 255540 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 346 Data size: 64822 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 346 Data size: 113262 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
               TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -195,7 +195,7 @@ 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 40
             Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
@@ -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: 5461 Data size: 901772 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1386 Data size: 228984 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: 5461 Data size: 901772 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1386 Data size: 228984 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: 2730 Data size: 510974 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 693 Data size: 129752 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: 2730 Data size: 510974 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 693 Data size: 129752 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: 1365 Data size: 255540 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 346 Data size: 64822 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 346 Data size: 113262 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
               TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -518,7 +518,7 @@ 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: 1365 Data size: 446640 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 346 Data size: 113262 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 40
             Statistics: Num rows: 40 Data size: 13206 Basic stats: COMPLETE Column stats: COMPLETE
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 1d4a41c..c943d7b 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_14.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_14.q.out
@@ -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: 606 Data size: 105558 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 758 Data size: 132082 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: 606 Data size: 105558 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 758 Data size: 132082 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: 303 Data size: 52846 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 379 Data size: 66108 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: 303 Data size: 52846 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 379 Data size: 66108 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: 151 Data size: 26432 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 189 Data size: 33008 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: 151 Data size: 36700 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 189 Data size: 45860 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: 151 Data size: 36700 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 189 Data size: 45860 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: 151 Data size: 36700 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 189 Data size: 45860 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 151 Data size: 36700 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 189 Data size: 45860 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 6a038a3..ae684e6 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_16.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_16.q.out
@@ -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: 4096 Data size: 769522 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 1154088 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: 4096 Data size: 769522 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 1154088 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: 2048 Data size: 303516 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3072 Data size: 455172 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: 2048 Data size: 303516 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3072 Data size: 455172 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: 1024 Data size: 151758 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1536 Data size: 227586 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: 1024 Data size: 307406 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 1024 Data size: 307406 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1536 Data size: 461058 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 4a8eb4d..6adb128 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_17.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_17.q.out
@@ -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: 4096 Data size: 549274 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6141 Data size: 823456 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: 4096 Data size: 1212930 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6141 Data size: 1818460 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: 4096 Data size: 1212930 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6141 Data size: 1818460 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: 4096 Data size: 1212930 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6141 Data size: 1818460 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 4096 Data size: 1212930 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6141 Data size: 1818460 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_7.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_7.q.out
index adfbdf9..d1a1ae4 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_7.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_7.q.out
@@ -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: 5461 Data size: 1342196 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 11033 Data size: 2711364 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11033 Data size: 1865892 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -118,7 +118,7 @@ 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 25
             Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
@@ -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: 5461 Data size: 1342196 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 11033 Data size: 2711364 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11033 Data size: 1865892 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Map Vectorization:
@@ -337,7 +337,7 @@ 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: 5461 Data size: 923616 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 11033 Data size: 1865892 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 25
             Statistics: Num rows: 25 Data size: 4380 Basic stats: COMPLETE Column stats: COMPLETE
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 6a038a3..ae684e6 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_9.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_9.q.out
@@ -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: 4096 Data size: 769522 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 6144 Data size: 1154088 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: 4096 Data size: 769522 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6144 Data size: 1154088 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: 2048 Data size: 303516 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3072 Data size: 455172 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: 2048 Data size: 303516 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 3072 Data size: 455172 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: 1024 Data size: 151758 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1536 Data size: 227586 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: 1024 Data size: 307406 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1536 Data size: 461058 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 1024 Data size: 307406 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1536 Data size: 461058 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_div0.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_div0.q.out
index bcd4bdb..1aa85b8 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_div0.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_div0.q.out
@@ -211,7 +211,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterLongColGreaterLongScalar(col 3:bigint, val 0), FilterLongColLessLongScalar(col 3:bigint, val 100000000))
               predicate: ((cbigint > 0L) and (cbigint < 100000000L)) (type: boolean)
-              Statistics: Num rows: 1365 Data size: 16320 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 3215 Data size: 38416 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: (cbigint - 988888L) (type: bigint), (cdouble / UDFToDouble((cbigint - 988888L))) (type: double), (1.2 / CAST( (cbigint - 988888L) AS decimal(19,0))) (type: decimal(22,21))
                 outputColumnNames: _col0, _col1, _col2
@@ -220,7 +220,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [13, 16, 19]
                     selectExpressions: LongColSubtractLongScalar(col 3:bigint, val 988888) -> 13:bigint, DoubleColDivideDoubleColumn(col 5:double, col 15:double)(children: CastLongToDouble(col 14:bigint)(children: LongColSubtractLongScalar(col 3:bigint, val 988888) -> 14:bigint) -> 15:double) -> 16:double, DecimalScalarDivideDecimalColumn(val 1.2, col 18:decimal(19,0))(children: CastLongToDecimal(col 17:bigint)(children: LongColSubtractLongScalar(col 3:bigint, val 988888) -> 17:bigint) - [...]
-                Statistics: Num rows: 1365 Data size: 174720 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3215 Data size: 411520 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: bigint), _col1 (type: double)
                   sort order: ++
@@ -229,7 +229,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: 1365 Data size: 174720 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 3215 Data size: 411520 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
                   value expressions: _col2 (type: decimal(22,21))
       Execution mode: vectorized
@@ -250,7 +250,7 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: bigint), KEY.reducesinkkey1 (type: double), VALUE._col0 (type: decimal(22,21))
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 1365 Data size: 174720 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 3215 Data size: 411520 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 100
             Statistics: Num rows: 100 Data size: 12800 Basic stats: COMPLETE Column stats: COMPLETE
@@ -414,7 +414,7 @@ STAGE PLANS:
                   native: true
                   predicateExpression: FilterExprAndExpr(children: FilterDoubleColGreaterEqualDoubleScalar(col 5:double, val -500.0), FilterDoubleColLessDoubleScalar(col 5:double, val -199.0))
               predicate: ((cdouble >= -500.0D) and (cdouble < -199.0D)) (type: boolean)
-              Statistics: Num rows: 1365 Data size: 16320 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 256 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: (cdouble + 200.0D) (type: double), (UDFToDouble(cbigint) / (cdouble + 200.0D)) (type: double), ((cdouble + 200.0D) / (cdouble + 200.0D)) (type: double), (3.0D / (cdouble + 200.0D)) (type: double), (1.2D / (cdouble + 200.0D)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col4, _col5
@@ -423,7 +423,7 @@ STAGE PLANS:
                     native: true
                     projectedOutputColumnNums: [13, 16, 19, 21, 23]
                     selectExpressions: DoubleColAddDoubleScalar(col 5:double, val 200.0) -> 13:double, DoubleColDivideDoubleColumn(col 14:double, col 15:double)(children: CastLongToDouble(col 3:bigint) -> 14:double, DoubleColAddDoubleScalar(col 5:double, val 200.0) -> 15:double) -> 16:double, DoubleColDivideDoubleColumn(col 17:double, col 18:double)(children: DoubleColAddDoubleScalar(col 5:double, val 200.0) -> 17:double, DoubleColAddDoubleScalar(col 5:double, val 200.0) -> 18:double) -> [...]
-                Statistics: Num rows: 1365 Data size: 65520 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 20 Data size: 960 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: double), _col1 (type: double)
                   sort order: ++
@@ -432,7 +432,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: 1365 Data size: 65520 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 20 Data size: 960 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
                   value expressions: _col2 (type: double), _col4 (type: double), _col5 (type: double)
       Execution mode: vectorized
@@ -453,13 +453,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: double), KEY.reducesinkkey1 (type: double), VALUE._col0 (type: double), KEY.reducesinkkey1 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1365 Data size: 65520 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 20 Data size: 960 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 100
-            Statistics: Num rows: 100 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 960 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 100 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 960 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_limit.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_limit.q.out
index 67a9983..ad3b1ac 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_limit.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_limit.q.out
@@ -25,11 +25,11 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 183488 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((cint > 0) and (UDFToDouble(cbigint) < cdouble)) (type: boolean)
-              Statistics: Num rows: 1365 Data size: 20400 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2048 Data size: 30600 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cbigint (type: bigint), cdouble (type: double)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1365 Data size: 16320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2048 Data size: 24480 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 7
                   Statistics: Num rows: 7 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/parquet_vectorization_offset_limit.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_offset_limit.q.out
index e204b9e..a384b7c 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_offset_limit.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_offset_limit.q.out
@@ -25,11 +25,11 @@ STAGE PLANS:
             Statistics: Num rows: 12288 Data size: 183488 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((cint > 0) and (UDFToDouble(cbigint) < cdouble)) (type: boolean)
-              Statistics: Num rows: 1365 Data size: 20400 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2048 Data size: 30600 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: cbigint (type: bigint), cdouble (type: double)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1365 Data size: 16320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2048 Data size: 24480 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 2
                   Offset of rows: 3
diff --git a/ql/src/test/results/clientpositive/pcr.q.out b/ql/src/test/results/clientpositive/pcr.q.out
index 584a4c4..54a9585 100644
--- a/ql/src/test/results/clientpositive/pcr.q.out
+++ b/ql/src/test/results/clientpositive/pcr.q.out
@@ -80,16 +80,16 @@ STAGE PLANS:
             Filter Operator
               isSamplingPred: false
               predicate: (key < 5) (type: boolean)
-              Statistics: Num rows: 13 Data size: 3614 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 11 Data size: 3058 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), value (type: string), ds (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 13 Data size: 3614 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11 Data size: 3058 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col2 (type: string)
                   null sort order: zz
                   sort order: ++
-                  Statistics: Num rows: 13 Data size: 3614 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11 Data size: 3058 Basic stats: COMPLETE Column stats: COMPLETE
                   tag: -1
                   value expressions: _col1 (type: string)
                   auto parallelism: false
@@ -201,13 +201,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string)
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 13 Data size: 3614 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 11 Data size: 3058 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
-            Statistics: Num rows: 13 Data size: 3614 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 11 Data size: 3058 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -287,16 +287,16 @@ STAGE PLANS:
             Filter Operator
               isSamplingPred: false
               predicate: ((key < 5) or (ds <= '2000-04-09')) (type: boolean)
-              Statistics: Num rows: 40 Data size: 11120 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 36 Data size: 10008 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 36 Data size: 3384 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   null sort order: z
                   sort order: +
-                  Statistics: Num rows: 40 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 36 Data size: 3384 Basic stats: COMPLETE Column stats: COMPLETE
                   tag: -1
                   value expressions: _col1 (type: string)
                   auto parallelism: false
@@ -457,13 +457,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 40 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 36 Data size: 3384 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
-            Statistics: Num rows: 40 Data size: 3760 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 36 Data size: 3384 Basic stats: COMPLETE Column stats: COMPLETE
... 3635 lines suppressed ...