You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2018/06/08 06:00:04 UTC

[01/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Repository: hive
Updated Branches:
  refs/heads/master 5aa8f8776 -> 13960aa99


http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/standalone-metastore/src/test/java/org/apache/hadoop/hive/common/ndv/hll/TestHyperLogLogMerge.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/common/ndv/hll/TestHyperLogLogMerge.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/common/ndv/hll/TestHyperLogLogMerge.java
new file mode 100644
index 0000000..2007c6f
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/common/ndv/hll/TestHyperLogLogMerge.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.common.ndv.hll;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestHyperLogLogMerge {
+  // 5% tolerance for estimated count
+  private float longRangeTolerance = 5.0f;
+  private float shortRangeTolerance = 2.0f;
+
+  int size;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][] {
+      { 1_000 }, { 10_000 }, { 100_000 }, { 1_000_000 }, { 10_000_000 }
+      // { 100_000_000 }, { 1_000_000_000 } 1B passed but is super slow
+    });
+  }
+
+  public TestHyperLogLogMerge(int size) {
+    this.size = size;
+  }
+
+  @Test
+  public void testHLLMergeDisjoint() {
+    HyperLogLog hll1 = HyperLogLog.builder().setNumRegisterIndexBits(16).build();
+    for (int i = 0; i < size; i++) {
+      hll1.addLong(i);
+    }
+    HyperLogLog hll2 = HyperLogLog.builder().setNumRegisterIndexBits(16).build();
+    for (int i = size; i < 2 * size; i++) {
+      hll2.addLong(i);
+    }
+    hll1.merge(hll2);
+    double threshold = size > 40000 ? longRangeTolerance : shortRangeTolerance;
+    double delta = threshold * size / 100;
+    long expected = 2 * size;
+    long actual = hll1.count();
+    assertEquals(expected, actual, delta);
+  }
+
+  @Test
+  public void testHLLMerge25PercentOverlap() {
+    HyperLogLog hll1 = HyperLogLog.builder().setNumRegisterIndexBits(16).build();
+    for (int i = 0; i < size; i++) {
+      hll1.addLong(i);
+    }
+    HyperLogLog hll2 = HyperLogLog.builder().setNumRegisterIndexBits(16).build();
+    int start = (int) (0.75 * size);
+    int end = (int) (size * 1.75);
+    for (int i = start; i < end; i++) {
+      hll2.addLong(i);
+    }
+    hll1.merge(hll2);
+    double threshold = size > 40000 ? longRangeTolerance : shortRangeTolerance;
+    double delta = threshold * size / 100;
+    long expected = (long) (1.75 * size);
+    long actual = hll1.count();
+    assertEquals(expected, actual, delta);
+  }
+
+  @Test
+  public void testHLLMerge50PercentOverlap() {
+    HyperLogLog hll1 = HyperLogLog.builder().setNumRegisterIndexBits(16).build();
+    for (int i = 0; i < size; i++) {
+      hll1.addLong(i);
+    }
+    HyperLogLog hll2 = HyperLogLog.builder().setNumRegisterIndexBits(16).build();
+    int start = (int) (0.5 * size);
+    int end = (int) (size * 1.5);
+    for (int i = start; i < end; i++) {
+      hll2.addLong(i);
+    }
+    hll1.merge(hll2);
+    double threshold = size > 40000 ? longRangeTolerance : shortRangeTolerance;
+    double delta = threshold * size / 100;
+    long expected = (long) (1.5 * size);
+    long actual = hll1.count();
+    assertEquals(expected, actual, delta);
+  }
+
+
+  @Test
+  public void testHLLMerge75PercentOverlap() {
+    HyperLogLog hll1 = HyperLogLog.builder().setNumRegisterIndexBits(16).build();
+    for (int i = 0; i < size; i++) {
+      hll1.addLong(i);
+    }
+    HyperLogLog hll2 = HyperLogLog.builder().setNumRegisterIndexBits(16).build();
+    int start = (int) (0.25 * size);
+    int end = (int) (size * 1.25);
+    for (int i = start; i < end; i++) {
+      hll2.addLong(i);
+    }
+    hll1.merge(hll2);
+    double threshold = size > 40000 ? longRangeTolerance : shortRangeTolerance;
+    double delta = threshold * size / 100;
+    long expected = (long) (1.25 * size);
+    long actual = hll1.count();
+    assertEquals(expected, actual, delta);
+  }
+
+
+  @Test
+  public void testHLLMerge100PercentOverlap() {
+    HyperLogLog hll1 = HyperLogLog.builder().setNumRegisterIndexBits(16).build();
+    for (int i = 0; i < size; i++) {
+      hll1.addLong(i);
+    }
+    HyperLogLog hll2 = HyperLogLog.builder().setNumRegisterIndexBits(16).build();
+    for (int i = 0; i < size; i++) {
+      hll2.addLong(i);
+    }
+    hll1.merge(hll2);
+    double threshold = size > 40000 ? longRangeTolerance : shortRangeTolerance;
+    double delta = threshold * size / 100;
+    long expected = size;
+    long actual = hll1.count();
+    assertEquals(expected, actual, delta);
+  }
+
+}


[14/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

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


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

Branch: refs/heads/master
Commit: 13960aa999aeeba5105fd64fef79dc6ed2fbe220
Parents: 5aa8f87
Author: Gopal V <go...@apache.org>
Authored: Sat Dec 2 21:20:00 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Thu Jun 7 22:59:43 2018 -0700

----------------------------------------------------------------------
 .../clientpositive/autoColumnStats_2.q.out      |   16 +-
 .../clientpositive/autoColumnStats_9.q.out      |    4 +-
 .../test/results/clientpositive/bitvector.q.out |    2 +-
 .../clientpositive/compute_stats_date.q.out     |    2 +-
 .../confirm_initial_tbl_stats.q.out             |   14 +-
 .../clientpositive/cross_join_merge.q.out       |   80 +-
 .../results/clientpositive/describe_table.q.out |    4 +-
 .../encrypted/encryption_move_tbl.q.out         |    4 +-
 ql/src/test/results/clientpositive/hll.q.out    |    6 +-
 .../clientpositive/llap/autoColumnStats_2.q.out |   16 +-
 .../clientpositive/llap/auto_join1.q.out        |    8 +-
 .../clientpositive/llap/auto_join21.q.out       |   10 +-
 .../clientpositive/llap/auto_join29.q.out       |   90 +-
 .../clientpositive/llap/auto_join30.q.out       |   16 +-
 .../llap/auto_sortmerge_join_6.q.out            |  558 ++--
 .../clientpositive/llap/bucket_groupby.q.out    |   16 +-
 .../llap/bucket_map_join_tez1.q.out             |  198 +-
 .../llap/bucket_map_join_tez2.q.out             |  312 +-
 .../clientpositive/llap/check_constraint.q.out  |    4 +-
 .../llap/correlationoptimizer1.q.out            |   86 +-
 .../llap/correlationoptimizer2.q.out            |   20 +-
 .../llap/correlationoptimizer3.q.out            |   68 +-
 .../llap/correlationoptimizer6.q.out            |  148 +-
 .../clientpositive/llap/cross_join.q.out        |    8 +-
 .../llap/dynamic_semijoin_reduction.q.out       |    8 +-
 .../llap/dynamic_semijoin_reduction_sw.q.out    |    8 +-
 .../llap/dynamic_semijoin_user_level.q.out      |    8 +-
 .../clientpositive/llap/explainanalyze_2.q.out  | 1455 ++++-----
 .../clientpositive/llap/explainuser_1.q.out     |   44 +-
 .../clientpositive/llap/explainuser_2.q.out     | 2788 +++++++++---------
 .../clientpositive/llap/explainuser_4.q.out     |   14 +-
 .../results/clientpositive/llap/groupby1.q.out  |    6 +-
 .../results/clientpositive/llap/groupby2.q.out  |   10 +-
 .../llap/groupby_resolution.q.out               |   16 +-
 .../llap/hybridgrace_hashjoin_1.q.out           |   12 +-
 .../llap/hybridgrace_hashjoin_2.q.out           |   24 +-
 .../results/clientpositive/llap/join1.q.out     |    8 +-
 .../clientpositive/llap/join32_lessSize.q.out   |  173 +-
 .../llap/join_max_hashtable.q.out               |   32 +-
 .../llap/limit_join_transpose.q.out             |    8 +-
 .../clientpositive/llap/limit_pushdown.q.out    |   14 +-
 .../clientpositive/llap/limit_pushdown3.q.out   |   34 +-
 .../llap/llap_vector_nohybridgrace.q.out        |    4 +-
 .../clientpositive/llap/llapdecider.q.out       |   64 +-
 .../clientpositive/llap/mapjoin_decimal.q.out   |    8 +-
 .../clientpositive/llap/mapjoin_mapjoin.q.out   |   18 +-
 .../test/results/clientpositive/llap/mrr.q.out  |   42 +-
 .../clientpositive/llap/multiMapJoin2.q.out     |   58 +-
 .../llap/offset_limit_ppd_optimizer.q.out       |   18 +-
 .../clientpositive/llap/partialdhj.q.out        |    8 +-
 .../clientpositive/llap/quotedid_smb.q.out      |    4 +-
 .../llap/reduce_deduplicate_extended.q.out      |   58 +-
 .../clientpositive/llap/results_cache_1.q.out   |    4 +-
 .../llap/results_cache_lifetime.q.out           |    2 +-
 .../results/clientpositive/llap/semijoin.q.out  |    4 +-
 .../clientpositive/llap/semijoin_hint.q.out     |   26 +-
 .../clientpositive/llap/sharedworkext.q.out     |   62 +-
 .../results/clientpositive/llap/skewjoin.q.out  |   14 +-
 .../clientpositive/llap/smb_mapjoin_6.q.out     |   12 +-
 .../clientpositive/llap/subquery_corr.q.out     |   12 +-
 .../clientpositive/llap/subquery_exists.q.out   |    8 +-
 .../clientpositive/llap/subquery_in.q.out       |   52 +-
 .../llap/subquery_in_having.q.out               |   32 +-
 .../clientpositive/llap/subquery_multi.q.out    |   24 +-
 .../clientpositive/llap/subquery_notin.q.out    |   36 +-
 .../clientpositive/llap/subquery_scalar.q.out   |   16 +-
 .../clientpositive/llap/subquery_select.q.out   |   12 +-
 .../clientpositive/llap/subquery_views.q.out    |   18 +-
 .../llap/tez_bmj_schema_evolution.q.out         |    8 +-
 .../llap/tez_dynpart_hashjoin_2.q.out           |   24 +-
 .../clientpositive/llap/tez_join_hash.q.out     |   22 +-
 .../clientpositive/llap/tez_join_tests.q.out    |   12 +-
 .../clientpositive/llap/tez_joins_explain.q.out |   12 +-
 .../clientpositive/llap/tez_smb_main.q.out      |    4 +-
 .../results/clientpositive/llap/tez_union.q.out |   64 +-
 .../clientpositive/llap/tez_union2.q.out        |   16 +-
 .../llap/tez_union_multiinsert.q.out            |   24 +-
 .../llap/tez_vector_dynpart_hashjoin_2.q.out    |   24 +-
 .../clientpositive/llap/unionDistinct_1.q.out   |   38 +-
 .../clientpositive/llap/unionDistinct_3.q.out   |   48 +-
 .../clientpositive/llap/union_fast_stats.q.out  |    8 +-
 .../clientpositive/llap/union_top_level.q.out   |   16 +-
 .../llap/vector_cast_constant.q.out             |   12 +-
 .../llap/vector_count_distinct.q.out            |    6 +-
 .../llap/vector_groupby_mapjoin.q.out           |    4 +-
 .../llap/vector_groupby_reduce.q.out            |   26 +-
 .../llap/vector_left_outer_join.q.out           |    4 +-
 .../llap/vector_llap_text_1.q.out               |    8 +-
 .../clientpositive/llap/vectorization_0.q.out   |   14 +-
 .../llap/vectorization_div0.q.out               |    8 +-
 .../vectorization_input_format_excludes.q.out   |   40 +-
 .../llap/vectorization_limit.q.out              |   34 +-
 .../llap/vectorization_short_regress.q.out      |   26 +-
 .../llap/vectorized_context.q.out               |   10 +-
 .../llap/vectorized_distinct_gby.q.out          |    8 +-
 .../llap/vectorized_mapjoin.q.out               |    4 +-
 .../llap/vectorized_nested_mapjoin.q.out        |    6 +-
 .../llap/vectorized_parquet.q.out               |   10 +-
 .../llap/vectorized_shufflejoin.q.out           |    4 +-
 .../clientpositive/mapjoin_mapjoin.q.out        |   30 +-
 .../spark/bucket_map_join_tez1.q.out            |   76 +-
 .../spark/bucket_map_join_tez2.q.out            |  208 +-
 .../clientpositive/spark/join32_lessSize.q.out  |   90 +-
 .../clientpositive/spark/mapjoin_mapjoin.q.out  |   26 +-
 .../spark/spark_explainuser_1.q.out             |   44 +-
 .../clientpositive/tez/explainanalyze_1.q.out   |   14 +-
 .../clientpositive/tez/explainanalyze_3.q.out   |    4 +-
 .../clientpositive/tez/explainanalyze_4.q.out   |   12 +-
 .../clientpositive/tez/explainanalyze_5.q.out   |    6 +-
 .../clientpositive/tez/explainuser_3.q.out      |    4 +-
 .../tez/hybridgrace_hashjoin_1.q.out            |   12 +-
 .../tez/hybridgrace_hashjoin_2.q.out            |   24 +-
 .../results/clientpositive/tez/tez-tag.q.out    |    2 +-
 .../ndv/NumDistinctValueEstimatorFactory.java   |    7 +-
 .../hive/common/ndv/hll/HLLDenseRegister.java   |   25 +
 .../hive/common/ndv/hll/HLLSparseRegister.java  |   13 +
 .../hadoop/hive/common/ndv/hll/HyperLogLog.java |   63 +-
 .../hive/common/ndv/hll/HyperLogLogUtils.java   |   19 +-
 .../hive/common/ndv/hll/TestHyperLogLog.java    |  124 +-
 .../common/ndv/hll/TestHyperLogLogMerge.java    |  147 +
 120 files changed, 4444 insertions(+), 4018 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/autoColumnStats_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/autoColumnStats_2.q.out b/ql/src/test/results/clientpositive/autoColumnStats_2.q.out
index b9c6835..854c779 100644
--- a/ql/src/test/results/clientpositive/autoColumnStats_2.q.out
+++ b/ql/src/test/results/clientpositive/autoColumnStats_2.q.out
@@ -129,7 +129,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -148,7 +148,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -253,7 +253,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -272,7 +272,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	307                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	6.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	7                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -301,7 +301,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	10                  	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -320,7 +320,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	10                  	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	307                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	6.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	7                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -349,7 +349,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	20                  	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -368,7 +368,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	10                  	 	 	 	 	 	 	 	 	 	 
-distinct_count      	319                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	8.0                 	 	 	 	 	 	 	 	 	 	 
 max_col_len         	8                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/autoColumnStats_9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/autoColumnStats_9.q.out b/ql/src/test/results/clientpositive/autoColumnStats_9.q.out
index 898598f..f2a8cdc 100644
--- a/ql/src/test/results/clientpositive/autoColumnStats_9.q.out
+++ b/ql/src/test/results/clientpositive/autoColumnStats_9.q.out
@@ -253,7 +253,7 @@ data_type           	int
 min                 	0                   	 	 	 	 	 	 	 	 	 	 
 max                 	498                 	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	303                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 max_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -272,7 +272,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	307                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	6.834630350194552   	 	 	 	 	 	 	 	 	 	 
 max_col_len         	7                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/bitvector.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bitvector.q.out b/ql/src/test/results/clientpositive/bitvector.q.out
index 1e0284f..e310ebe 100644
--- a/ql/src/test/results/clientpositive/bitvector.q.out
+++ b/ql/src/test/results/clientpositive/bitvector.q.out
@@ -9,7 +9,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/compute_stats_date.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/compute_stats_date.q.out b/ql/src/test/results/clientpositive/compute_stats_date.q.out
index 35c0f8d..58d2d70 100644
--- a/ql/src/test/results/clientpositive/compute_stats_date.q.out
+++ b/ql/src/test/results/clientpositive/compute_stats_date.q.out
@@ -43,7 +43,7 @@ POSTHOOK: query: select compute_stats(fl_date, 'hll') from tab_date
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tab_date
 #### A masked pattern was here ####
-{"columntype":"Date","min":"2000-11-20","max":"2010-10-29","countnulls":0,"numdistinctvalues":19,"ndvbitvector":HLL�ê����!���]����������������Y���T��P��R������W����ĈN������������}
+{"columntype":"Date","min":"2000-11-20","max":"2010-10-29","countnulls":0,"numdistinctvalues":19,"ndvbitvector":HLL�Ǫ����!���]����������������Y���T���P��R������W����ĈN������������}
 PREHOOK: query: explain
 analyze table tab_date compute statistics for columns fl_date
 PREHOOK: type: ANALYZE_TABLE

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/confirm_initial_tbl_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/confirm_initial_tbl_stats.q.out b/ql/src/test/results/clientpositive/confirm_initial_tbl_stats.q.out
index a900194..3d64168 100644
--- a/ql/src/test/results/clientpositive/confirm_initial_tbl_stats.q.out
+++ b/ql/src/test/results/clientpositive/confirm_initial_tbl_stats.q.out
@@ -19,7 +19,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -105,7 +105,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	307                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	6.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	7                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -134,7 +134,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	430                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	431                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	6.802               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	7                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -163,7 +163,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	307                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	6.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	7                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -199,7 +199,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -237,7 +237,7 @@ data_type           	tinyint
 min                 	-64                 	 	 	 	 	 	 	 	 	 	 
 max                 	62                  	 	 	 	 	 	 	 	 	 	 
 num_nulls           	3115                	 	 	 	 	 	 	 	 	 	 
-distinct_count      	127                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	130                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 max_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -275,7 +275,7 @@ data_type           	timestamp
 min                 	-30                 	 	 	 	 	 	 	 	 	 	 
 max                 	31                  	 	 	 	 	 	 	 	 	 	 
 num_nulls           	3115                	 	 	 	 	 	 	 	 	 	 
-distinct_count      	35                  	 	 	 	 	 	 	 	 	 	 
+distinct_count      	36                  	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 max_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/cross_join_merge.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cross_join_merge.q.out b/ql/src/test/results/clientpositive/cross_join_merge.q.out
index 84d78f3..3389638 100644
--- a/ql/src/test/results/clientpositive/cross_join_merge.q.out
+++ b/ql/src/test/results/clientpositive/cross_join_merge.q.out
@@ -233,7 +233,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain
 select src1.key from src src1 join src src2 on 5 = src2.key join src src3 on src1.key=src3.key
 PREHOOK: type: QUERY
@@ -250,7 +250,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: src3
+            alias: src1
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: key is not null (type: boolean)
@@ -260,29 +260,34 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  sort order: 
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: string)
           TableScan
-            alias: src2
+            alias: src3
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (UDFToDouble(key) = 5.0D) (type: boolean)
-              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                expressions: key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  sort order: 
-                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 
-            1 
+            0 _col0 (type: string)
+            1 _col0 (type: string)
           outputColumnNames: _col0
-          Statistics: Num rows: 125000 Data size: 2781000 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             table:
@@ -295,45 +300,36 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col0 (type: string)
-              sort order: +
-              Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 125000 Data size: 2781000 Basic stats: COMPLETE Column stats: NONE
+              sort order: 
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string)
           TableScan
-            alias: src1
+            alias: src2
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
-              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              predicate: (UDFToDouble(key) = 5.0D) (type: boolean)
+              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: key (type: string)
-                outputColumnNames: _col0
-                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col0 (type: string)
-                  sort order: +
-                  Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  sort order: 
+                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 _col0 (type: string)
-            1 _col0 (type: string)
-          outputColumnNames: _col2
-          Statistics: Num rows: 137500 Data size: 3059100 Basic stats: COMPLETE Column stats: NONE
-          Select Operator
-            expressions: _col2 (type: string)
-            outputColumnNames: _col0
-            Statistics: Num rows: 137500 Data size: 3059100 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 137500 Data size: 3059100 Basic stats: COMPLETE Column stats: NONE
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            0 
+            1 
+          outputColumnNames: _col0
+          Statistics: Num rows: 137500 Data size: 3059050 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 137500 Data size: 3059050 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/describe_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/describe_table.q.out b/ql/src/test/results/clientpositive/describe_table.q.out
index d88424f..8c7a16c 100644
--- a/ql/src/test/results/clientpositive/describe_table.q.out
+++ b/ql/src/test/results/clientpositive/describe_table.q.out
@@ -206,7 +206,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -304,7 +304,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
index a63cd86..b73f3ac 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
@@ -66,7 +66,7 @@ data_type           	int
 min                 	0                   	 	 	 	 	 	 	 	 	 	 
 max                 	498                 	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	303                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 max_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -85,7 +85,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	307                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	6.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	7                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/hll.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/hll.q.out b/ql/src/test/results/clientpositive/hll.q.out
index 12f6a36..4b445ea 100644
--- a/ql/src/test/results/clientpositive/hll.q.out
+++ b/ql/src/test/results/clientpositive/hll.q.out
@@ -182,7 +182,7 @@ data_type           	int
 min                 	0                   	 	 	 	 	 	 	 	 	 	 
 max                 	498                 	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	303                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 max_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -236,7 +236,7 @@ data_type           	double
 min                 	0.0                 	 	 	 	 	 	 	 	 	 	 
 max                 	498.0               	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 max_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -290,7 +290,7 @@ data_type           	decimal(10,0)
 min                 	0                   	 	 	 	 	 	 	 	 	 	 
 max                 	498                 	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 max_col_len         	                    	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/autoColumnStats_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/autoColumnStats_2.q.out b/ql/src/test/results/clientpositive/llap/autoColumnStats_2.q.out
index 83c538b..7f0867a 100644
--- a/ql/src/test/results/clientpositive/llap/autoColumnStats_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/autoColumnStats_2.q.out
@@ -127,7 +127,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -146,7 +146,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -251,7 +251,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -270,7 +270,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	0                   	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	307                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	6.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	7                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -299,7 +299,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	10                  	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -318,7 +318,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	10                  	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	307                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	6.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	7                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -347,7 +347,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	20                  	 	 	 	 	 	 	 	 	 	 
-distinct_count      	309                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	2.812               	 	 	 	 	 	 	 	 	 	 
 max_col_len         	3                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 
@@ -366,7 +366,7 @@ data_type           	string
 min                 	                    	 	 	 	 	 	 	 	 	 	 
 max                 	                    	 	 	 	 	 	 	 	 	 	 
 num_nulls           	10                  	 	 	 	 	 	 	 	 	 	 
-distinct_count      	319                 	 	 	 	 	 	 	 	 	 	 
+distinct_count      	316                 	 	 	 	 	 	 	 	 	 	 
 avg_col_len         	8.0                 	 	 	 	 	 	 	 	 	 	 
 max_col_len         	8                   	 	 	 	 	 	 	 	 	 	 
 num_trues           	                    	 	 	 	 	 	 	 	 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/auto_join1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/auto_join1.q.out b/ql/src/test/results/clientpositive/llap/auto_join1.q.out
index 61cedb7..17987a7 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join1.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join1.q.out
@@ -69,14 +69,14 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col2
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: UDFToInteger(_col0) (type: int), _col2 (type: string)
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.TextInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -85,7 +85,7 @@ STAGE PLANS:
                           Select Operator
                             expressions: _col0 (type: int), _col1 (type: string)
                             outputColumnNames: key, value
-                            Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
                               aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                               mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/auto_join21.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/auto_join21.q.out b/ql/src/test/results/clientpositive/llap/auto_join21.q.out
index 64e6422..5889720 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join21.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join21.q.out
@@ -75,25 +75,25 @@ STAGE PLANS:
                   1 key (type: string)
                   2 key (type: string)
                 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     sort order: ++++++
-                    Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/auto_join29.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/auto_join29.q.out b/ql/src/test/results/clientpositive/llap/auto_join29.q.out
index c357c1f..3891ea0 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join29.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join29.q.out
@@ -75,25 +75,25 @@ STAGE PLANS:
                   1 key (type: string)
                   2 key (type: string)
                 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     sort order: ++++++
-                    Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -693,25 +693,25 @@ STAGE PLANS:
                   1 key (type: string)
                   2 key (type: string)
                 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 137 Data size: 73158 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 137 Data size: 73158 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     sort order: ++++++
-                    Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 137 Data size: 73158 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 137 Data size: 73158 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 137 Data size: 73158 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1311,25 +1311,25 @@ STAGE PLANS:
                   1 key (type: string)
                   2 key (type: string)
                 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 432 Data size: 230688 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 432 Data size: 230688 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     sort order: ++++++
-                    Statistics: Num rows: 432 Data size: 230688 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 432 Data size: 230688 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 432 Data size: 230688 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1938,25 +1938,25 @@ STAGE PLANS:
                   1 key (type: string)
                   2 key (type: string)
                 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     sort order: ++++++
-                    Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 434 Data size: 231756 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 415 Data size: 221610 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2567,25 +2567,25 @@ STAGE PLANS:
                   1 key (type: string)
                   2 key (type: string)
                 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     sort order: ++++++
-                    Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2685,25 +2685,25 @@ STAGE PLANS:
                   1 key (type: string)
                   2 key (type: string)
                 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 136 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 136 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     sort order: ++++++
-                    Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 136 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 136 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 136 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3263,15 +3263,15 @@ STAGE PLANS:
                     input vertices:
                       1 Map 3
                       2 Map 4
-                    Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 136 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                      Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 136 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                         sort order: ++++++
-                        Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 136 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -3312,10 +3312,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 136 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 136 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3394,15 +3394,15 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 4
-                      Statistics: Num rows: 431 Data size: 230154 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 431 Data size: 230154 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                           sort order: ++++++
-                          Statistics: Num rows: 431 Data size: 230154 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -3427,10 +3427,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 431 Data size: 230154 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 431 Data size: 230154 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3511,15 +3511,15 @@ STAGE PLANS:
                       input vertices:
                         1 Map 3
                         2 Map 4
-                      Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                           sort order: ++++++
-                          Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -3560,10 +3560,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 143 Data size: 76362 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/auto_join30.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/auto_join30.q.out b/ql/src/test/results/clientpositive/llap/auto_join30.q.out
index 19d6c55..1f95237 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join30.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join30.q.out
@@ -67,7 +67,7 @@ STAGE PLANS:
                         outputColumnNames: _col2, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: sum(hash(_col2,_col3))
                           mode: hash
@@ -167,7 +167,7 @@ STAGE PLANS:
                       outputColumnNames: _col2, _col3
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(hash(_col2,_col3))
                         mode: hash
@@ -300,7 +300,7 @@ STAGE PLANS:
                       outputColumnNames: _col2, _col3
                       input vertices:
                         0 Map 1
-                      Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: sum(hash(_col2,_col3))
                         mode: hash
@@ -431,7 +431,7 @@ STAGE PLANS:
                         input vertices:
                           0 Map 1
                           2 Map 4
-                        Statistics: Num rows: 1309 Data size: 233002 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1251 Data size: 222678 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: sum(hash(_col2,_col3))
                           mode: hash
@@ -605,7 +605,7 @@ STAGE PLANS:
                   1 _col0 (type: string)
                   2 _col0 (type: string)
                 outputColumnNames: _col2, _col3
-                Statistics: Num rows: 1309 Data size: 233002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1251 Data size: 222678 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(hash(_col2,_col3))
                   mode: hash
@@ -758,7 +758,7 @@ STAGE PLANS:
                   1 _col0 (type: string)
                   2 _col0 (type: string)
                 outputColumnNames: _col2, _col3
-                Statistics: Num rows: 1309 Data size: 233002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1251 Data size: 222678 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(hash(_col2,_col3))
                   mode: hash
@@ -911,7 +911,7 @@ STAGE PLANS:
                   1 _col0 (type: string)
                   2 _col0 (type: string)
                 outputColumnNames: _col2, _col3
-                Statistics: Num rows: 1309 Data size: 233002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1251 Data size: 222678 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(hash(_col2,_col3))
                   mode: hash
@@ -1064,7 +1064,7 @@ STAGE PLANS:
                   1 _col0 (type: string)
                   2 _col0 (type: string)
                 outputColumnNames: _col2, _col3
-                Statistics: Num rows: 1309 Data size: 233002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1251 Data size: 222678 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(hash(_col2,_col3))
                   mode: hash


[09/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
----------------------------------------------------------------------
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 8801331..0ebd5ca 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -1713,9 +1713,9 @@ Stage-0
     Stage-1
       Reducer 2 llap
       File Output Operator [FS_19]
-        Select Operator [SEL_18] (rows=366 width=178)
+        Select Operator [SEL_18] (rows=365 width=178)
           Output:["_col0","_col1"]
-          Filter Operator [FIL_17] (rows=366 width=179)
+          Filter Operator [FIL_17] (rows=365 width=179)
             predicate:_col3 is null
             Merge Join Operator [MERGEJOIN_22] (rows=500 width=179)
               Conds:RS_14._col1=RS_15._col0(Left Outer),Output:["_col0","_col1","_col3"]
@@ -1777,9 +1777,9 @@ Stage-0
     Stage-1
       Reducer 3 llap
       File Output Operator [FS_18]
-        Select Operator [SEL_17] (rows=183 width=178)
+        Select Operator [SEL_17] (rows=185 width=178)
           Output:["_col0","_col1"]
-          Filter Operator [FIL_16] (rows=183 width=179)
+          Filter Operator [FIL_16] (rows=185 width=179)
             predicate:_col4 is null
             Merge Join Operator [MERGEJOIN_21] (rows=250 width=179)
               Conds:RS_13._col0, _col1=RS_14._col0, _col1(Left Outer),Output:["_col0","_col1","_col4"]
@@ -1852,7 +1852,7 @@ Stage-0
     Stage-1
       Reducer 2 llap
       File Output Operator [FS_12]
-        Merge Join Operator [MERGEJOIN_17] (rows=133 width=178)
+        Merge Join Operator [MERGEJOIN_17] (rows=131 width=178)
           Conds:RS_8._col0, _col1=RS_9._col0, _col1(Left Semi),Output:["_col0","_col1"]
         <-Map 1 [SIMPLE_EDGE] llap
           SHUFFLE [RS_8]
@@ -1904,7 +1904,7 @@ Stage-0
     Stage-1
       Reducer 2 llap
       File Output Operator [FS_12]
-        Merge Join Operator [MERGEJOIN_17] (rows=133 width=178)
+        Merge Join Operator [MERGEJOIN_17] (rows=131 width=178)
           Conds:RS_8._col0, _col1=RS_9._col0, _col1(Left Semi),Output:["_col0","_col1"]
         <-Map 1 [SIMPLE_EDGE] llap
           SHUFFLE [RS_8]
@@ -1946,7 +1946,7 @@ Stage-0
     Stage-1
       Reducer 2 llap
       File Output Operator [FS_12]
-        Merge Join Operator [MERGEJOIN_17] (rows=133 width=178)
+        Merge Join Operator [MERGEJOIN_17] (rows=131 width=178)
           Conds:RS_8._col0=RS_9._col0(Left Semi),Output:["_col0","_col1"]
         <-Map 1 [SIMPLE_EDGE] llap
           SHUFFLE [RS_8]
@@ -2067,16 +2067,16 @@ Stage-0
         <-Reducer 3 [SIMPLE_EDGE] llap
           SHUFFLE [RS_27]
             PartitionCols:_col2
-            Filter Operator [FIL_37] (rows=66 width=186)
+            Filter Operator [FIL_37] (rows=65 width=186)
               predicate:_col2 is not null
-              Group By Operator [GBY_14] (rows=66 width=186)
+              Group By Operator [GBY_14] (rows=65 width=186)
                 Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
               <-Reducer 2 [SIMPLE_EDGE] llap
                 SHUFFLE [RS_13]
                   PartitionCols:_col0, _col1
-                  Group By Operator [GBY_12] (rows=66 width=186)
+                  Group By Operator [GBY_12] (rows=65 width=186)
                     Output:["_col0","_col1","_col2"],aggregations:["count()"],keys:_col0, _col1
-                    Merge Join Operator [MERGEJOIN_43] (rows=133 width=178)
+                    Merge Join Operator [MERGEJOIN_43] (rows=131 width=178)
                       Conds:RS_8._col0=RS_9._col0(Left Semi),Output:["_col0","_col1"]
                     <-Map 5 [SIMPLE_EDGE] llap
                       SHUFFLE [RS_9]
@@ -2667,7 +2667,7 @@ Stage-0
                     PartitionCols:_col0
                     Group By Operator [GBY_10] (rows=16 width=94)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                      Merge Join Operator [MERGEJOIN_24] (rows=40 width=86)
+                      Merge Join Operator [MERGEJOIN_24] (rows=39 width=86)
                         Conds:RS_6._col0=RS_7._col0(Inner),Output:["_col0"]
                       <-Map 1 [SIMPLE_EDGE] llap
                         SHUFFLE [RS_6]
@@ -2728,7 +2728,7 @@ Stage-0
                     PartitionCols:_col0
                     Group By Operator [GBY_10] (rows=16 width=94)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                      Merge Join Operator [MERGEJOIN_24] (rows=40 width=86)
+                      Merge Join Operator [MERGEJOIN_24] (rows=39 width=86)
                         Conds:RS_6._col0=RS_7._col0(Inner),Output:["_col0"]
                       <-Map 1 [SIMPLE_EDGE] llap
                         SHUFFLE [RS_6]
@@ -2789,7 +2789,7 @@ Stage-0
                     PartitionCols:_col0
                     Group By Operator [GBY_10] (rows=16 width=94)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                      Map Join Operator [MAPJOIN_24] (rows=40 width=86)
+                      Map Join Operator [MAPJOIN_24] (rows=39 width=86)
                         Conds:RS_6._col0=SEL_5._col0(Inner),Output:["_col0"]
                       <-Map 1 [BROADCAST_EDGE] llap
                         BROADCAST [RS_6]
@@ -2838,16 +2838,16 @@ Stage-0
           PARTITION_ONLY_SHUFFLE [RS_18]
             Group By Operator [GBY_17] (rows=1 width=16)
               Output:["_col0","_col1"],aggregations:["sum(_col0)","sum(_col1)"]
-              Select Operator [SEL_15] (rows=10 width=94)
+              Select Operator [SEL_15] (rows=9 width=94)
                 Output:["_col0","_col1"]
-                Group By Operator [GBY_14] (rows=10 width=94)
+                Group By Operator [GBY_14] (rows=9 width=94)
                   Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
                 <-Reducer 2 [SIMPLE_EDGE] llap
                   SHUFFLE [RS_13]
                     PartitionCols:_col0
-                    Group By Operator [GBY_12] (rows=10 width=94)
+                    Group By Operator [GBY_12] (rows=9 width=94)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                      Merge Join Operator [MERGEJOIN_26] (rows=20 width=86)
+                      Merge Join Operator [MERGEJOIN_26] (rows=19 width=86)
                         Conds:RS_8._col0=RS_9._col0(Left Semi),Output:["_col0"]
                       <-Map 1 [SIMPLE_EDGE] llap
                         SHUFFLE [RS_8]
@@ -5325,9 +5325,9 @@ Stage-3
                     <-Reducer 2 [CUSTOM_SIMPLE_EDGE] llap
                       File Output Operator [FS_11]
                         table:{"name:":"default.dest_j1_n16"}
-                        Select Operator [SEL_9] (rows=809 width=95)
+                        Select Operator [SEL_9] (rows=791 width=95)
                           Output:["_col0","_col1"]
-                          Merge Join Operator [MERGEJOIN_16] (rows=809 width=178)
+                          Merge Join Operator [MERGEJOIN_16] (rows=791 width=178)
                             Conds:RS_6._col0=RS_7._col0(Inner),Output:["_col0","_col2"]
                           <-Map 1 [SIMPLE_EDGE] llap
                             SHUFFLE [RS_6]
@@ -5349,7 +5349,7 @@ Stage-3
                                     default@src,src2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
                       PARTITION_ONLY_SHUFFLE [RS_2]
                         PartitionCols:rand()
-                        Select Operator [SEL_1] (rows=809 width=95)
+                        Select Operator [SEL_1] (rows=791 width=95)
                           Output:["key","value"]
                            Please refer to the previous Select Operator [SEL_9]
 
@@ -5569,7 +5569,7 @@ Stage-0
     Stage-1
       Reducer 2 llap
       File Output Operator [FS_10]
-        Merge Join Operator [MERGEJOIN_15] (rows=809 width=356)
+        Merge Join Operator [MERGEJOIN_15] (rows=791 width=356)
           Conds:RS_6._col0=RS_7._col0(Inner),Output:["_col0","_col1","_col2","_col3"]
         <-Map 1 [SIMPLE_EDGE] llap
           SHUFFLE [RS_6]


[12/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
index 66b019b..2416384 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
@@ -235,25 +235,25 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
                     sort order: +++
-                    Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -823,15 +823,15 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
                             sort order: +++
-                            Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -840,10 +840,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1436,7 +1436,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 207 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 204 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -1565,7 +1565,7 @@ STAGE PLANS:
                     1 _col0 (type: int)
                   input vertices:
                     1 Map 4
-                  Statistics: Num rows: 207 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 204 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count()
                     mode: hash
@@ -1696,7 +1696,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 708 Data size: 5664 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 689 Data size: 5512 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -1731,16 +1731,16 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col1
-                Statistics: Num rows: 414 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: int)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 414 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 1632 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: 414 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 408 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -1838,11 +1838,11 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 4
-                        Statistics: Num rows: 414 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: int)
                           outputColumnNames: _col0
-                          Statistics: Num rows: 414 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 408 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -1851,7 +1851,7 @@ STAGE PLANS:
                               1 _col0 (type: int)
                             input vertices:
                               0 Map 1
-                            Statistics: Num rows: 708 Data size: 5664 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 689 Data size: 5512 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
                               aggregations: count()
                               mode: hash
@@ -2007,7 +2007,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 708 Data size: 5664 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 689 Data size: 5512 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -2042,16 +2042,16 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col1
-                Statistics: Num rows: 414 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: int)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 414 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 1632 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: 414 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 408 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -2149,11 +2149,11 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 4
-                        Statistics: Num rows: 414 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: int)
                           outputColumnNames: _col0
-                          Statistics: Num rows: 414 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 408 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -2162,7 +2162,7 @@ STAGE PLANS:
                               1 _col0 (type: int)
                             input vertices:
                               0 Map 1
-                            Statistics: Num rows: 708 Data size: 5664 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 689 Data size: 5512 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
                               aggregations: count()
                               mode: hash
@@ -2566,11 +2566,11 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 414 Data size: 39330 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 38760 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: int), substr(_col2, 5) (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 414 Data size: 39330 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 38760 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col1)
                     keys: _col0 (type: int)
@@ -2708,11 +2708,11 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2
                         input vertices:
                           0 Map 2
-                        Statistics: Num rows: 414 Data size: 39330 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 38760 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: int), substr(_col2, 5) (type: string)
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 414 Data size: 39330 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 408 Data size: 38760 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
                             aggregations: sum(_col1)
                             keys: _col0 (type: int)
@@ -2866,14 +2866,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col1 (type: int)
                 outputColumnNames: _col1, _col2, _col3
-                Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col3 (type: int), _col2 (type: double), _col1 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2949,14 +2949,14 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2, _col3
                         input vertices:
                           1 Reducer 3
-                        Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col3 (type: int), _col2 (type: double), _col1 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3139,14 +3139,14 @@ STAGE PLANS:
                   1 _col0 (type: int)
                   2 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col4
-                Statistics: Num rows: 655 Data size: 121830 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 646 Data size: 120156 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 655 Data size: 121830 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 646 Data size: 120156 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 655 Data size: 121830 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 646 Data size: 120156 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3202,14 +3202,14 @@ STAGE PLANS:
                         input vertices:
                           1 Map 2
                           2 Map 3
-                        Statistics: Num rows: 655 Data size: 121830 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 646 Data size: 120156 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: string), _col4 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 655 Data size: 121830 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 646 Data size: 120156 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 655 Data size: 121830 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 646 Data size: 120156 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3304,12 +3304,12 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 414 Data size: 39330 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 38760 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: 414 Data size: 39330 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 408 Data size: 38760 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -3362,14 +3362,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 654 Data size: 121644 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 654 Data size: 121644 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 654 Data size: 121644 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3422,7 +3422,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 414 Data size: 39330 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 38760 Basic stats: COMPLETE Column stats: COMPLETE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
@@ -3432,14 +3432,14 @@ STAGE PLANS:
                           outputColumnNames: _col0, _col1, _col3
                           input vertices:
                             1 Map 3
-                          Statistics: Num rows: 654 Data size: 121644 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE
                           Select Operator
                             expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                             outputColumnNames: _col0, _col1, _col2
-                            Statistics: Num rows: 654 Data size: 121644 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE
                             File Output Operator
                               compressed: false
-                              Statistics: Num rows: 654 Data size: 121644 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE
                               table:
                                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3586,14 +3586,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 207 Data size: 21321 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 204 Data size: 21012 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: double), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 207 Data size: 21321 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 204 Data size: 21012 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 207 Data size: 21321 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 204 Data size: 21012 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3675,14 +3675,14 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Reducer 2
-                        Statistics: Num rows: 207 Data size: 21321 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 204 Data size: 21012 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: double), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 207 Data size: 21321 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 204 Data size: 21012 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 207 Data size: 21321 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 204 Data size: 21012 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3799,14 +3799,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: double), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3882,14 +3882,14 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Reducer 2
-                        Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: double), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 261 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 258 Data size: 26574 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3987,10 +3987,10 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4071,10 +4071,10 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4178,14 +4178,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4258,14 +4258,14 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4364,7 +4364,7 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col2
-                Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 814 Data size: 6512 Basic stats: COMPLETE Column stats: COMPLETE
                 Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -4374,14 +4374,14 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col2
                   input vertices:
                     1 Map 4
-                  Statistics: Num rows: 1385 Data size: 11080 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1375 Data size: 11000 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: int), _col2 (type: int)
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1385 Data size: 11080 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1375 Data size: 11000 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
-                      Statistics: Num rows: 1385 Data size: 11080 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1375 Data size: 11000 Basic stats: COMPLETE Column stats: COMPLETE
                       table:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4430,7 +4430,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col2
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 814 Data size: 6512 Basic stats: COMPLETE Column stats: COMPLETE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
@@ -4440,14 +4440,14 @@ STAGE PLANS:
                           outputColumnNames: _col0, _col2
                           input vertices:
                             1 Map 3
-                          Statistics: Num rows: 1385 Data size: 11080 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1375 Data size: 11000 Basic stats: COMPLETE Column stats: COMPLETE
                           Select Operator
                             expressions: _col0 (type: int), _col2 (type: int)
                             outputColumnNames: _col0, _col1
-                            Statistics: Num rows: 1385 Data size: 11080 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 1375 Data size: 11000 Basic stats: COMPLETE Column stats: COMPLETE
                             File Output Operator
                               compressed: false
-                              Statistics: Num rows: 1385 Data size: 11080 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 1375 Data size: 11000 Basic stats: COMPLETE Column stats: COMPLETE
                               table:
                                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4585,14 +4585,14 @@ STAGE PLANS:
                   0 _col0 (type: int), _col2 (type: string)
                   1 _col0 (type: int), _col2 (type: string)
                 outputColumnNames: _col0, _col1, _col4
-                Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4689,14 +4689,14 @@ STAGE PLANS:
                   0 _col0 (type: int), _col2 (type: string)
                   1 _col0 (type: int), _col2 (type: string)
                 outputColumnNames: _col0, _col1, _col4
-                Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 414 Data size: 77004 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -4774,14 +4774,14 @@ STAGE PLANS:
                   0 _col0 (type: int), _col1 (type: string)
                   1 _col0 (type: int), _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -5349,14 +5349,14 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -5970,14 +5970,14 @@ STAGE PLANS:
                   0 _col0 (type: int), _col1 (type: string)
                   1 _col0 (type: int), _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -6545,14 +6545,14 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
-                            Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 394 Data size: 73284 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
index 47079f4..7aea667 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
@@ -178,7 +178,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -194,11 +194,18 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -214,7 +221,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 4 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: c
@@ -240,16 +247,16 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                  0 _col1 (type: string)
+                  1 _col1 (type: string)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 814 Data size: 6512 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col1 (type: string)
+                  key expressions: _col0 (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col1 (type: string)
-                  Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: int)
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 814 Data size: 6512 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col2 (type: int)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -257,17 +264,17 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
-                  1 _col1 (type: string)
-                outputColumnNames: _col0, _col3
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), _col3 (type: int)
+                  expressions: _col0 (type: int), _col2 (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -292,8 +299,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 3 (CUSTOM_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -308,41 +315,12 @@ STAGE PLANS:
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col0 (type: int)
-                          1 _col0 (type: int)
-                        outputColumnNames: _col0, _col1
-                        input vertices:
-                          1 Map 3
-                        Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col1 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col1 (type: string)
-                          Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
-                          value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -365,6 +343,25 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 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: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -374,15 +371,32 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: string)
                   1 _col1 (type: string)
-                outputColumnNames: _col0, _col3
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 814 Data size: 6512 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: 814 Data size: 6512 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col2 (type: int)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), _col3 (type: int)
+                  expressions: _col0 (type: int), _col2 (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -496,14 +510,14 @@ STAGE PLANS:
                   0 UDFToDouble(_col0) (type: double)
                   1 UDFToDouble(_col0) (type: double)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 382 Data size: 71052 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 382 Data size: 71052 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 382 Data size: 71052 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -585,14 +599,14 @@ STAGE PLANS:
                   0 UDFToDouble(_col0) (type: double)
                   1 UDFToDouble(_col0) (type: double)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 382 Data size: 71052 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 382 Data size: 71052 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 382 Data size: 71052 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -670,10 +684,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -724,10 +738,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -826,10 +840,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -880,10 +894,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -982,10 +996,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1055,10 +1069,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 267 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 272 Data size: 2176 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1141,10 +1155,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 191 Data size: 1528 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 200 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 191 Data size: 1528 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 200 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1214,10 +1228,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           0 Reducer 2
-                        Statistics: Num rows: 191 Data size: 1528 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 200 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 191 Data size: 1528 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 200 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1565,7 +1579,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int), _col1 (type: string)
                   1 _col0 (type: int), _col1 (type: string)
-                Statistics: Num rows: 195 Data size: 1560 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 197 Data size: 1576 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -1675,7 +1689,7 @@ STAGE PLANS:
                     1 _col0 (type: int), _col1 (type: string)
                   input vertices:
                     1 Map 4
-                  Statistics: Num rows: 195 Data size: 1560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 197 Data size: 1576 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count()
                     mode: hash
@@ -1893,8 +1907,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 3 (CUSTOM_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1909,41 +1923,12 @@ STAGE PLANS:
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col0 (type: int)
-                          1 _col0 (type: int)
-                        outputColumnNames: _col0, _col1
-                        input vertices:
-                          1 Map 3
-                        Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col1 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col1 (type: string)
-                          Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
-                          value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -1966,6 +1951,25 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 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: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -1975,15 +1979,32 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: string)
                   1 _col1 (type: string)
-                outputColumnNames: _col0, _col3
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 814 Data size: 6512 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: 814 Data size: 6512 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col2 (type: int)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), _col3 (type: int)
+                  expressions: _col0 (type: int), _col2 (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2010,7 +2031,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2026,11 +2047,18 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2046,7 +2074,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 4 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: c
@@ -2072,16 +2100,16 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                  0 _col1 (type: string)
+                  1 _col1 (type: string)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 814 Data size: 6512 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col1 (type: string)
+                  key expressions: _col0 (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col1 (type: string)
-                  Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: int)
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 814 Data size: 6512 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col2 (type: int)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2089,17 +2117,17 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
-                  1 _col1 (type: string)
-                outputColumnNames: _col0, _col3
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), _col3 (type: int)
+                  expressions: _col0 (type: int), _col2 (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/check_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/check_constraint.q.out b/ql/src/test/results/clientpositive/llap/check_constraint.q.out
index fda46c9..085b003 100644
--- a/ql/src/test/results/clientpositive/llap/check_constraint.q.out
+++ b/ql/src/test/results/clientpositive/llap/check_constraint.q.out
@@ -1536,11 +1536,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 268 Data size: 71020 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 262 Data size: 69430 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: UDFToInteger(_col0) (type: int), CAST( _col2 AS decimal(5,2)) (type: decimal(5,2)), _col1 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 268 Data size: 55476 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 262 Data size: 54234 Basic stats: COMPLETE Column stats: COMPLETE
                   Limit
                     Number of rows: 10
                     Statistics: Num rows: 10 Data size: 2070 Basic stats: COMPLETE Column stats: COMPLETE


[06/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/llapdecider.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/llapdecider.q.out b/ql/src/test/results/clientpositive/llap/llapdecider.q.out
index c7b2810..64ed44d 100644
--- a/ql/src/test/results/clientpositive/llap/llapdecider.q.out
+++ b/ql/src/test/results/clientpositive/llap/llapdecider.q.out
@@ -383,11 +383,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col3 (type: string)
                   sort order: +
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
             Execution mode: vectorized
@@ -395,10 +395,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -477,11 +477,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col3 (type: string)
                   sort order: +
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -489,10 +489,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -570,11 +570,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col3 (type: string)
                   sort order: +
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -582,10 +582,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -661,11 +661,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col3 (type: string)
                   sort order: +
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
             Execution mode: vectorized
@@ -673,10 +673,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -752,11 +752,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col3 (type: string)
                   sort order: +
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
             Execution mode: vectorized
@@ -764,10 +764,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -845,11 +845,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col3 (type: string)
                   sort order: +
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
             Execution mode: vectorized
@@ -857,10 +857,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -936,11 +936,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col3 (type: string)
                   sort order: +
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
             Execution mode: vectorized
@@ -948,10 +948,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1030,11 +1030,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col3 (type: string)
                   sort order: +
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -1042,10 +1042,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out b/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
index 0bf0f94..795ecfe 100644
--- a/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
+++ b/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
@@ -112,11 +112,11 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 1102 Data size: 246848 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1112 Data size: 249088 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: decimal(4,2))
                           sort order: +
-                          Statistics: Num rows: 1102 Data size: 246848 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1112 Data size: 249088 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col1 (type: decimal(4,0))
             Execution mode: llap
             LLAP IO: all inputs
@@ -145,10 +145,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: decimal(4,2)), VALUE._col0 (type: decimal(4,0))
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1102 Data size: 246848 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1112 Data size: 249088 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1102 Data size: 246848 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1112 Data size: 249088 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/mapjoin_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mapjoin_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/mapjoin_mapjoin.q.out
index 8433e56..0ef6bbb 100644
--- a/ql/src/test/results/clientpositive/llap/mapjoin_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/mapjoin_mapjoin.q.out
@@ -580,9 +580,9 @@ STAGE PLANS:
                         condition map:
                              Inner Join 0 to 1
                         keys:
-                          0 _col1 (type: string)
+                          0 _col0 (type: string)
                           1 _col0 (type: string)
-                        outputColumnNames: _col0, _col2
+                        outputColumnNames: _col1, _col2
                         input vertices:
                           1 Map 3
                         Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE
@@ -590,7 +590,7 @@ STAGE PLANS:
                           condition map:
                                Inner Join 0 to 1
                           keys:
-                            0 _col0 (type: string)
+                            0 _col1 (type: string)
                             1 _col0 (type: string)
                           outputColumnNames: _col2
                           input vertices:
@@ -613,13 +613,13 @@ STAGE PLANS:
         Map 3 
             Map Operator Tree:
                 TableScan
-                  alias: src
+                  alias: src1
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: value is not null (type: boolean)
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: value (type: string)
+                      expressions: key (type: string)
                       outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
@@ -632,13 +632,13 @@ STAGE PLANS:
         Map 4 
             Map Operator Tree:
                 TableScan
-                  alias: src1
+                  alias: src
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: string)
+                      expressions: value (type: string)
                       outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/mrr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mrr.q.out b/ql/src/test/results/clientpositive/llap/mrr.q.out
index 737c738..cb25b8c 100644
--- a/ql/src/test/results/clientpositive/llap/mrr.q.out
+++ b/ql/src/test/results/clientpositive/llap/mrr.q.out
@@ -457,17 +457,17 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   keys: _col1 (type: string), _col2 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 70310 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)
-                    Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -475,27 +475,27 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(_col1)
                   keys: _col0 (type: string)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col1 (type: bigint), _col0 (type: string)
                     sort order: ++
-                    Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: bigint)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -882,17 +882,17 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           keys: _col1 (type: string), _col2 (type: string)
                           mode: hash
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 395 Data size: 70310 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)
-                            Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -902,27 +902,27 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(_col1)
                   keys: _col0 (type: string)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col1 (type: bigint), _col0 (type: string)
                     sort order: ++
-                    Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: bigint)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1360,15 +1360,15 @@ STAGE PLANS:
                   1 _col0 (type: string)
                   2 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3, _col4, _col5
-                Statistics: Num rows: 54 Data size: 10692 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 51 Data size: 10098 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col3 (type: bigint), _col1 (type: bigint), _col4 (type: string), _col5 (type: bigint)
                   outputColumnNames: _col0, _col1, _col3, _col4, _col5
-                  Statistics: Num rows: 54 Data size: 15390 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 51 Data size: 14535 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
-                    Statistics: Num rows: 54 Data size: 15390 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 51 Data size: 14535 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -1376,10 +1376,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 54 Data size: 15390 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 51 Data size: 14535 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 54 Data size: 15390 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 51 Data size: 14535 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/multiMapJoin2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/multiMapJoin2.q.out b/ql/src/test/results/clientpositive/llap/multiMapJoin2.q.out
index 9dfc408..61b7c4e 100644
--- a/ql/src/test/results/clientpositive/llap/multiMapJoin2.q.out
+++ b/ql/src/test/results/clientpositive/llap/multiMapJoin2.q.out
@@ -47,11 +47,11 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 4
-                        Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
-                          Statistics: Num rows: 80 Data size: 6960 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 78 Data size: 6786 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -94,11 +94,11 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 6
-                        Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
-                          Statistics: Num rows: 80 Data size: 6960 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 78 Data size: 6786 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 6 
@@ -126,10 +126,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 80 Data size: 6960 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 78 Data size: 6786 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 80 Data size: 6960 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 78 Data size: 6786 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -350,21 +350,21 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
-                  Statistics: Num rows: 80 Data size: 6960 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 78 Data size: 6786 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 80 Data size: 6960 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 78 Data size: 6786 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 80 Data size: 6960 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 78 Data size: 6786 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -379,11 +379,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
-                  Statistics: Num rows: 80 Data size: 6960 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 78 Data size: 6786 Basic stats: COMPLETE Column stats: COMPLETE
         Union 3 
             Vertex: Union 3
 
@@ -556,11 +556,11 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 6
-                        Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
                           sort order: +
-                          Statistics: Num rows: 52 Data size: 4512 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 51 Data size: 4425 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 6 
@@ -593,17 +593,17 @@ STAGE PLANS:
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
-                  Statistics: Num rows: 52 Data size: 4512 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 51 Data size: 4425 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 52 Data size: 4524 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 51 Data size: 4437 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 52 Data size: 4524 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 51 Data size: 4437 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -750,7 +750,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 5
-                        Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           keys: _col0 (type: string)
                           mode: hash
@@ -808,7 +808,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 5
-                        Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           keys: _col0 (type: string)
                           mode: hash
@@ -990,7 +990,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 5
-                        Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           keys: _col0 (type: string)
                           mode: hash
@@ -1048,7 +1048,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 5
-                        Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           keys: _col0 (type: string)
                           mode: hash
@@ -1230,7 +1230,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 5
-                        Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           keys: _col0 (type: string)
                           mode: hash
@@ -1454,7 +1454,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 5
-                        Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           keys: _col0 (type: string)
                           mode: hash
@@ -1857,17 +1857,17 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 5
-                        Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           keys: _col0 (type: string)
                           mode: hash
                           outputColumnNames: _col0
-                          Statistics: Num rows: 309 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 316 Data size: 27492 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: 309 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 316 Data size: 27492 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
@@ -1896,17 +1896,17 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 7
-                        Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           keys: _col0 (type: string)
                           mode: hash
                           outputColumnNames: _col0
-                          Statistics: Num rows: 309 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 316 Data size: 27492 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: 309 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 316 Data size: 27492 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 6 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/offset_limit_ppd_optimizer.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/offset_limit_ppd_optimizer.q.out b/ql/src/test/results/clientpositive/llap/offset_limit_ppd_optimizer.q.out
index 6646027..ca0de47 100644
--- a/ql/src/test/results/clientpositive/llap/offset_limit_ppd_optimizer.q.out
+++ b/ql/src/test/results/clientpositive/llap/offset_limit_ppd_optimizer.q.out
@@ -403,12 +403,12 @@ STAGE PLANS:
                       keys: cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: double)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: double)
-                        Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.3
             Execution mode: llap
             LLAP IO: all inputs
@@ -419,7 +419,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Offset of rows: 10
@@ -518,7 +518,7 @@ STAGE PLANS:
                   keys: _col0 (type: tinyint)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                   Limit
                     Number of rows: 20
                     Offset of rows: 10
@@ -617,7 +617,7 @@ STAGE PLANS:
                   keys: _col0 (type: tinyint)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                   Limit
                     Number of rows: 20
                     Offset of rows: 10
@@ -713,7 +713,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 128 Data size: 2436 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 2492 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Offset of rows: 10
@@ -937,7 +937,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: complete
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 30591 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 307 Data size: 30393 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Offset of rows: 10
@@ -1339,7 +1339,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: complete
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Offset of rows: 10
@@ -1383,7 +1383,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: complete
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Offset of rows: 20

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/partialdhj.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/partialdhj.q.out b/ql/src/test/results/clientpositive/llap/partialdhj.q.out
index 207d94d..fde5496 100644
--- a/ql/src/test/results/clientpositive/llap/partialdhj.q.out
+++ b/ql/src/test/results/clientpositive/llap/partialdhj.q.out
@@ -143,11 +143,11 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 input vertices:
                   0 Reducer 3
-                Statistics: Num rows: 25 Data size: 6675 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 26 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 HybridGraceHashJoin: true
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 25 Data size: 6675 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 26 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -336,11 +336,11 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 input vertices:
                   1 Reducer 5
-                Statistics: Num rows: 25 Data size: 6675 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 26 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 HybridGraceHashJoin: true
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 25 Data size: 6675 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 26 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/quotedid_smb.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/quotedid_smb.q.out b/ql/src/test/results/clientpositive/llap/quotedid_smb.q.out
index 9c271a7..13f2f9a 100644
--- a/ql/src/test/results/clientpositive/llap/quotedid_smb.q.out
+++ b/ql/src/test/results/clientpositive/llap/quotedid_smb.q.out
@@ -89,10 +89,10 @@ STAGE PLANS:
                           0 _col1 (type: string)
                           1 _col1 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3
-                        Statistics: Num rows: 268 Data size: 95408 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 270 Data size: 96120 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 268 Data size: 95408 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 270 Data size: 96120 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/reduce_deduplicate_extended.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/reduce_deduplicate_extended.q.out b/ql/src/test/results/clientpositive/llap/reduce_deduplicate_extended.q.out
index ea2587f..dec81dd 100644
--- a/ql/src/test/results/clientpositive/llap/reduce_deduplicate_extended.q.out
+++ b/ql/src/test/results/clientpositive/llap/reduce_deduplicate_extended.q.out
@@ -42,10 +42,10 @@ STAGE PLANS:
                   keys: _col0 (type: string)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -313,18 +313,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(_col0)
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 20 Data size: 3720 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3534 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: 20 Data size: 3720 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 19 Data size: 3534 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: double)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -334,14 +334,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 20 Data size: 3720 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 19 Data size: 3534 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col2 (type: double)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 20 Data size: 1900 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 1805 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 20 Data size: 1900 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 19 Data size: 1805 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -419,21 +419,21 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   sort order: ++
-                  Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2596,10 +2596,10 @@ STAGE PLANS:
                   keys: _col0 (type: string)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2779,10 +2779,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: complete
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2860,12 +2860,12 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 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: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2874,14 +2874,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: complete
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 40 Data size: 7440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 7254 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col2 (type: double)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 40 Data size: 3800 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 3705 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 40 Data size: 3800 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 3705 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2959,21 +2959,21 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   sort order: ++
-                  Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3033,10 +3033,10 @@ STAGE PLANS:
                     keys: _col0 (type: string)
                     mode: complete
                     outputColumnNames: _col0
-                    Statistics: Num rows: 309 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 27492 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
-                      Statistics: Num rows: 309 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 316 Data size: 27492 Basic stats: COMPLETE Column stats: COMPLETE
                       table:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3093,10 +3093,10 @@ STAGE PLANS:
                   keys: _col0 (type: string)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/results_cache_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/results_cache_1.q.out b/ql/src/test/results/clientpositive/llap/results_cache_1.q.out
index b1423b9..6f5bfc3 100644
--- a/ql/src/test/results/clientpositive/llap/results_cache_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/results_cache_1.q.out
@@ -64,7 +64,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
-                Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 6328 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -197,7 +197,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
-                Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 6328 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/results_cache_lifetime.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/results_cache_lifetime.q.out b/ql/src/test/results/clientpositive/llap/results_cache_lifetime.q.out
index eeed7e3..8b8907b 100644
--- a/ql/src/test/results/clientpositive/llap/results_cache_lifetime.q.out
+++ b/ql/src/test/results/clientpositive/llap/results_cache_lifetime.q.out
@@ -83,7 +83,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
-                Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 6328 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/semijoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/semijoin.q.out b/ql/src/test/results/clientpositive/llap/semijoin.q.out
index 73b9d55..e785005 100644
--- a/ql/src/test/results/clientpositive/llap/semijoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/semijoin.q.out
@@ -2856,12 +2856,12 @@ STAGE PLANS:
                     keys: _col0 (type: string), _col1 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 10609 Data size: 1888402 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 10918 Data size: 1943404 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: 10609 Data size: 1888402 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 10918 Data size: 1943404 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out b/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
index e6f2346..679916d 100644
--- a/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
+++ b/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
@@ -222,7 +222,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1000 Data size: 87000 Basic stats: PARTIAL Column stats: PARTIAL
                         Group By Operator
-                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=309)
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=316)
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2
                           Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL
@@ -272,7 +272,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=309)
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=316)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL
@@ -478,16 +478,16 @@ STAGE PLANS:
                   0 cstring (type: string)
                   1 value (type: string)
                 outputColumnNames: str
-                Statistics: Num rows: 4058 Data size: 353046 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4110 Data size: 357570 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: str (type: string)
                   sort order: +
                   Map-reduce partition columns: str (type: string)
-                  Statistics: Num rows: 4058 Data size: 353046 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4110 Data size: 357570 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: str (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 4058 Data size: 706092 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4110 Data size: 715140 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=5000)
                     mode: hash
@@ -506,9 +506,9 @@ STAGE PLANS:
                 keys:
                   0 str (type: string)
                   1 key1 (type: string)
-                Statistics: Num rows: 4463 Data size: 388350 Basic stats: PARTIAL Column stats: NONE
+                Statistics: Num rows: 4521 Data size: 393327 Basic stats: PARTIAL Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 4463 Data size: 388350 Basic stats: PARTIAL Column stats: NONE
+                  Statistics: Num rows: 4521 Data size: 393327 Basic stats: PARTIAL Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     mode: hash
@@ -1476,16 +1476,16 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col1
-                Statistics: Num rows: 4058 Data size: 353046 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4110 Data size: 357570 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col1 (type: string)
-                  Statistics: Num rows: 4058 Data size: 353046 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4110 Data size: 357570 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 4058 Data size: 706092 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4110 Data size: 715140 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=5000)
                     mode: hash
@@ -1504,7 +1504,7 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
-                Statistics: Num rows: 4463 Data size: 388350 Basic stats: PARTIAL Column stats: NONE
+                Statistics: Num rows: 4521 Data size: 393327 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -2190,7 +2190,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1000 Data size: 87000 Basic stats: PARTIAL Column stats: PARTIAL
                       Group By Operator
-                        aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=309)
+                        aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=316)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL
@@ -2238,7 +2238,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=309)
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=316)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL


[13/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_6.q.out b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_6.q.out
index 8ee591b..b13beab 100644
--- a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_6.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_6.q.out
@@ -84,23 +84,13 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: b
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Operator Tree:
-                TableScan
                   alias: a
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
@@ -110,20 +100,14 @@ STAGE PLANS:
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 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)
-                        outputColumnNames: _col1
-                        Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col1 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col1 (type: string)
-                          Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: llap
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Map 5 
             Map Operator Tree:
                 TableScan
@@ -143,6 +127,25 @@ STAGE PLANS:
                         Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 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: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -152,7 +155,23 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0
+                Statistics: Num rows: 814 Data size: 3256 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: 814 Data size: 3256 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -162,7 +181,7 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: bigint)
-        Reducer 3 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -211,23 +230,13 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: b
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Operator Tree:
-                TableScan
                   alias: a
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
@@ -237,20 +246,14 @@ STAGE PLANS:
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 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)
-                        outputColumnNames: _col1
-                        Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col1 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col1 (type: string)
-                          Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: llap
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Map 5 
             Map Operator Tree:
                 TableScan
@@ -270,6 +273,25 @@ STAGE PLANS:
                         Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 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: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -279,7 +301,23 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0
+                Statistics: Num rows: 814 Data size: 3256 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: 814 Data size: 3256 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -289,7 +327,7 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: bigint)
-        Reducer 3 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -338,23 +376,13 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: b
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Operator Tree:
-                TableScan
                   alias: a
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
@@ -364,20 +392,14 @@ STAGE PLANS:
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 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)
-                        outputColumnNames: _col1
-                        Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col1 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col1 (type: string)
-                          Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: llap
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Map 5 
             Map Operator Tree:
                 TableScan
@@ -397,6 +419,25 @@ STAGE PLANS:
                         Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 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: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -406,7 +447,23 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0
+                Statistics: Num rows: 814 Data size: 3256 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: 814 Data size: 3256 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -416,7 +473,7 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: bigint)
-        Reducer 3 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -465,23 +522,13 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: b
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Operator Tree:
-                TableScan
                   alias: a
                   Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
@@ -491,20 +538,14 @@ STAGE PLANS:
                       expressions: key (type: int)
                       outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 2000 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)
-                        outputColumnNames: _col0
-                        Statistics: Num rows: 809 Data size: 3236 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: 809 Data size: 3236 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: llap
+                      Reduce Output Operator
+                        key expressions: UDFToDouble(_col0) (type: double)
+                        sort order: +
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Map 5 
             Map Operator Tree:
                 TableScan
@@ -524,6 +565,25 @@ STAGE PLANS:
                         Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 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: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -533,7 +593,23 @@ STAGE PLANS:
                 keys:
                   0 UDFToDouble(_col0) (type: double)
                   1 UDFToDouble(_col0) (type: double)
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0
+                Statistics: Num rows: 791 Data size: 3164 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: 791 Data size: 3164 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 1305 Data size: 10440 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -543,7 +619,7 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: bigint)
-        Reducer 3 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -636,7 +712,7 @@ STAGE PLANS:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
                           2 _col0 (type: int)
-                        Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1361 Data size: 10888 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash
@@ -702,15 +778,15 @@ STAGE PLANS:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: b
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: c
+                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int)
+                      expressions: value (type: string)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -726,33 +802,33 @@ STAGE PLANS:
                         condition map:
                              Inner Join 0 to 1
                         keys:
-                          0 _col0 (type: int)
-                          1 _col0 (type: int)
-                        outputColumnNames: _col1
-                        Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
+                          0 _col1 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 814 Data size: 3256 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
-                          key expressions: _col1 (type: string)
+                          key expressions: _col0 (type: int)
                           sort order: +
-                          Map-reduce partition columns: _col1 (type: string)
-                          Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 814 Data size: 3256 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
         Map 5 
             Map Operator Tree:
                 TableScan
-                  alias: c
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: value is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: value (type: string)
+                      expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col0 (type: string)
+                        key expressions: _col0 (type: int)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -762,9 +838,9 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
-                  1 _col0 (type: string)
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -808,7 +884,7 @@ POSTHOOK: Input: default@tbl1_n4
 POSTHOOK: Input: default@tbl2_n3
 POSTHOOK: Input: default@tbl4
 #### A masked pattern was here ####
-2654
+90
 PREHOOK: query: explain select count(*) FROM tbl1_n4 a JOIN tbl2_n3 b ON a.key = b.key join src c on c.value = a.value
 PREHOOK: type: QUERY
 POSTHOOK: query: explain select count(*) FROM tbl1_n4 a JOIN tbl2_n3 b ON a.key = b.key join src c on c.value = a.value
@@ -823,23 +899,13 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: b
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Operator Tree:
-                TableScan
                   alias: a
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
@@ -849,20 +915,14 @@ STAGE PLANS:
                       expressions: key (type: int), value (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 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)
-                        outputColumnNames: _col1
-                        Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col1 (type: string)
-                          sort order: +
-                          Map-reduce partition columns: _col1 (type: string)
-                          Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: llap
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Map 5 
             Map Operator Tree:
                 TableScan
@@ -882,6 +942,25 @@ STAGE PLANS:
                         Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 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: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -891,7 +970,23 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0
+                Statistics: Num rows: 814 Data size: 3256 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: 814 Data size: 3256 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -901,7 +996,7 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: bigint)
-        Reducer 3 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -950,23 +1045,13 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: b
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Operator Tree:
-                TableScan
                   alias: a
                   Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
@@ -976,20 +1061,14 @@ STAGE PLANS:
                       expressions: key (type: int)
                       outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 2000 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)
-                        outputColumnNames: _col0
-                        Statistics: Num rows: 809 Data size: 3236 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: 809 Data size: 3236 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: llap
+                      Reduce Output Operator
+                        key expressions: UDFToDouble(_col0) (type: double)
+                        sort order: +
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Map 5 
             Map Operator Tree:
                 TableScan
@@ -1009,6 +1088,25 @@ STAGE PLANS:
                         Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 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: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -1018,7 +1116,23 @@ STAGE PLANS:
                 keys:
                   0 UDFToDouble(_col0) (type: double)
                   1 UDFToDouble(_col0) (type: double)
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0
+                Statistics: Num rows: 791 Data size: 3164 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: 791 Data size: 3164 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 1305 Data size: 10440 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -1028,7 +1142,7 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: bigint)
-        Reducer 3 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1121,7 +1235,7 @@ STAGE PLANS:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
                           2 _col0 (type: int)
-                        Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1361 Data size: 10888 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash
@@ -1187,15 +1301,15 @@ STAGE PLANS:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: b
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: c
+                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int)
+                      expressions: value (type: string)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1211,33 +1325,33 @@ STAGE PLANS:
                         condition map:
                              Inner Join 0 to 1
                         keys:
-                          0 _col0 (type: int)
-                          1 _col0 (type: int)
-                        outputColumnNames: _col1
-                        Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
+                          0 _col1 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 814 Data size: 3256 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
-                          key expressions: _col1 (type: string)
+                          key expressions: _col0 (type: int)
                           sort order: +
-                          Map-reduce partition columns: _col1 (type: string)
-                          Statistics: Num rows: 809 Data size: 73619 Basic stats: COMPLETE Column stats: COMPLETE
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 814 Data size: 3256 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
         Map 5 
             Map Operator Tree:
                 TableScan
-                  alias: c
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: value is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: value (type: string)
+                      expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col0 (type: string)
+                        key expressions: _col0 (type: int)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1247,9 +1361,9 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
-                  1 _col0 (type: string)
-                Statistics: Num rows: 1309 Data size: 10472 Basic stats: COMPLETE Column stats: COMPLETE
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 1343 Data size: 10744 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -1293,4 +1407,4 @@ POSTHOOK: Input: default@tbl1_n4
 POSTHOOK: Input: default@tbl2_n3
 POSTHOOK: Input: default@tbl4
 #### A masked pattern was here ####
-2654
+90

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out b/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
index 3a7d744..4925de5 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
@@ -666,12 +666,12 @@ STAGE PLANS:
                       keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: PARTIAL
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: PARTIAL
                         value expressions: _col1 (type: bigint)
             Execution mode: llap
             LLAP IO: no inputs
@@ -683,10 +683,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: PARTIAL
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: PARTIAL
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1051,12 +1051,12 @@ STAGE PLANS:
                       keys: key (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: PARTIAL
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: PARTIAL
                         value expressions: _col1 (type: bigint)
             Execution mode: llap
             LLAP IO: no inputs
@@ -1068,10 +1068,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: PARTIAL
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: PARTIAL
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat


[03/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vector_llap_text_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_llap_text_1.q.out b/ql/src/test/results/clientpositive/llap/vector_llap_text_1.q.out
index f7721b6..a35b816 100644
--- a/ql/src/test/results/clientpositive/llap/vector_llap_text_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_llap_text_1.q.out
@@ -233,7 +233,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col3
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 399 Data size: 74214 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
                           outputColumnNames: _col0, _col1, _col2
@@ -241,7 +241,7 @@ STAGE PLANS:
                               className: VectorSelectOperator
                               native: true
                               projectedOutputColumnNums: [0, 4, 1]
-                          Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 399 Data size: 74214 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
                             sort order: +++
@@ -251,7 +251,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
                                 valueColumnNums: []
-                            Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 399 Data size: 74214 Basic stats: COMPLETE Column stats: COMPLETE
                             TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -295,7 +295,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2]
-                Statistics: Num rows: 391 Data size: 72726 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 399 Data size: 74214 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 10
                   Limit Vectorization:

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorization_0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_0.q.out b/ql/src/test/results/clientpositive/llap/vectorization_0.q.out
index 3d00bbe..1382503 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_0.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_0.q.out
@@ -30981,13 +30981,13 @@ STAGE PLANS:
                       keys: cstring1 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 304 Data size: 23864 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 300 Data size: 23550 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 304 Data size: 23864 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 300 Data size: 23550 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
                         value expressions: _col1 (type: bigint)
                         auto parallelism: true
@@ -31055,16 +31055,16 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 304 Data size: 23864 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 300 Data size: 23550 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: bigint), _col0 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 304 Data size: 23864 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 300 Data size: 23550 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col1 (type: string)
                     null sort order: a
                     sort order: +
-                    Statistics: Num rows: 304 Data size: 23864 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 300 Data size: 23550 Basic stats: COMPLETE Column stats: COMPLETE
                     tag: -1
                     value expressions: _col0 (type: bigint)
                     auto parallelism: false
@@ -31075,13 +31075,13 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: bigint), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 304 Data size: 23864 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 300 Data size: 23550 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
                   NumFilesPerFileSink: 1
-                  Statistics: Num rows: 304 Data size: 23864 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 300 Data size: 23550 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorization_div0.q.out
----------------------------------------------------------------------
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 e00dcd9..61f1e26 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_div0.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_div0.q.out
@@ -687,7 +687,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: ((cdouble > 1.0E9D) or (cint > 500000000) or (ctinyint = 0Y)) (type: boolean)
-                    Statistics: Num rows: 4193 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4191 Data size: 75120 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
@@ -696,7 +696,7 @@ STAGE PLANS:
                           native: true
                           projectedOutputColumnNums: [2, 3, 0, 16, 17, 18, 19, 20, 14]
                           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 13:double, col 15:double)(children: CastLongToDouble(col 3:bigint) -> 13:double, CastLongToDouble(col 14:bigint)(children: LongColSubtractLongScalar(col 3:bigint, val 1018195815) -> 14:bigint) -> 15:double) -> 17:double, DoubleColDivideDoubleColumn(col 13:double, col 15:double)(children: CastLongToDouble(col 0:tinyint) -> 13:double, CastLongToDouble(col 0:tinyint) -> 15:double) -> 18:double, LongColModuloLongColumn(col 2:int, col 14:int)(children: LongColSubtractLongScalar(col 2:int, val 528534767) -> 14:int) -> 19:int, LongColModuloLongColumn(col 3:bigint, col 14:bigint)(children: LongColSubtractLongScalar(col 3:bigint, val 1018195815) -> 14:bigint) -> 20:bigint, LongCol
 ModuloLongColumn(col 0:tinyint, col 0:tinyint) -> 14:tinyint
-                      Statistics: Num rows: 4193 Data size: 217816 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4191 Data size: 217720 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: +++++++++
@@ -704,7 +704,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: 4193 Data size: 217816 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4191 Data size: 217720 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -733,7 +733,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 4193 Data size: 217816 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4191 Data size: 217720 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 100
                   Limit Vectorization:

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorization_input_format_excludes.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_input_format_excludes.q.out b/ql/src/test/results/clientpositive/llap/vectorization_input_format_excludes.q.out
index fc4b032..becc03c 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_input_format_excludes.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_input_format_excludes.q.out
@@ -181,12 +181,12 @@ STAGE PLANS:
                       keys: _col0 (type: tinyint)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                      Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: tinyint)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: tinyint)
-                        Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: double), _col5 (type: bigint), _col6 (type: double), _col7 (type: double), _col8 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs (cache only)
@@ -213,14 +213,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), (_col4 / _col5) (type: double), power(((_col6 - ((_col7 * _col7) / _col8)) / _col8), 0.5) (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 128 Data size: 4484 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 4588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 128 Data size: 4484 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 4588 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -512,12 +512,12 @@ STAGE PLANS:
                       keys: _col0 (type: tinyint)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                      Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: tinyint)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: tinyint)
-                        Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: double), _col5 (type: bigint), _col6 (type: double), _col7 (type: double), _col8 (type: bigint)
             Execution mode: llap
             LLAP IO: all inputs (cache only)
@@ -539,14 +539,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), (_col4 / _col5) (type: double), power(((_col6 - ((_col7 * _col7) / _col8)) / _col8), 0.5) (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 128 Data size: 4484 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 4588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 128 Data size: 4484 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 4588 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -843,12 +843,12 @@ STAGE PLANS:
                       keys: _col0 (type: tinyint)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                      Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: tinyint)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: tinyint)
-                        Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: double), _col5 (type: bigint), _col6 (type: double), _col7 (type: double), _col8 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs (cache only)
@@ -875,14 +875,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), (_col4 / _col5) (type: double), power(((_col6 - ((_col7 * _col7) / _col8)) / _col8), 0.5) (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 128 Data size: 4484 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 4588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 128 Data size: 4484 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 4588 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1222,12 +1222,12 @@ STAGE PLANS:
                       keys: _col0 (type: tinyint)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                      Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: tinyint)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: tinyint)
-                        Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: double), _col5 (type: bigint), _col6 (type: double), _col7 (type: double), _col8 (type: bigint)
             Execution mode: llap
             LLAP IO: all inputs
@@ -1249,14 +1249,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), (_col4 / _col5) (type: double), power(((_col6 - ((_col7 * _col7) / _col8)) / _col8), 0.5) (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 128 Data size: 4484 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 4588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 128 Data size: 4484 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 4588 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorization_limit.q.out
----------------------------------------------------------------------
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 c569e68..6ea5fb2 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_limit.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_limit.q.out
@@ -296,7 +296,7 @@ STAGE PLANS:
                       keys: _col0 (type: tinyint)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 128 Data size: 2436 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 131 Data size: 2492 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: tinyint)
                         sort order: +
@@ -307,7 +307,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
                             valueColumnNums: [1, 2]
-                        Statistics: Num rows: 128 Data size: 2436 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 131 Data size: 2492 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: double), _col2 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -355,7 +355,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 128 Data size: 2436 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 2492 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: tinyint), (_col1 / _col2) (type: double)
                   outputColumnNames: _col0, _col1
@@ -364,7 +364,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 3]
                       selectExpressions: DoubleColDivideLongColumn(col 1:double, col 2:bigint) -> 3:double
-                  Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: tinyint), _col1 (type: double)
                     sort order: ++
@@ -374,7 +374,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
                         valueColumnNums: []
-                    Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.3
         Reducer 3 
             Execution mode: vectorized, llap
@@ -399,7 +399,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1]
-                Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Limit Vectorization:
@@ -500,7 +500,7 @@ STAGE PLANS:
                       keys: ctinyint (type: tinyint)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 128 Data size: 388 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 131 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: tinyint)
                         sort order: +
@@ -512,7 +512,7 @@ STAGE PLANS:
                             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
                             partitionColumnNums: [0]
                             valueColumnNums: []
-                        Statistics: Num rows: 128 Data size: 388 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 131 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.3
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -558,7 +558,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 128 Data size: 388 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Limit Vectorization:
@@ -731,7 +731,7 @@ STAGE PLANS:
                   keys: _col0 (type: tinyint)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: tinyint), _col1 (type: bigint)
                     sort order: ++
@@ -741,7 +741,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
                         valueColumnNums: []
-                    Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.3
         Reducer 3 
             Execution mode: vectorized, llap
@@ -766,7 +766,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1]
-                Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Limit Vectorization:
@@ -897,7 +897,7 @@ STAGE PLANS:
                       keys: cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 4157 Data size: 58088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4127 Data size: 57672 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: double)
                         sort order: +
@@ -908,7 +908,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
                             valueColumnNums: [1]
-                        Statistics: Num rows: 4157 Data size: 58088 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4127 Data size: 57672 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -956,7 +956,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 4157 Data size: 58088 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4127 Data size: 57672 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: bigint), _col0 (type: double)
                   sort order: ++
@@ -966,7 +966,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
                       valueColumnNums: []
-                  Statistics: Num rows: 4157 Data size: 58088 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 4127 Data size: 57672 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.3
         Reducer 3 
             Execution mode: vectorized, llap
@@ -991,7 +991,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [1, 0]
-                Statistics: Num rows: 4157 Data size: 58088 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 4127 Data size: 57672 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Limit Vectorization:

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorization_short_regress.q.out
----------------------------------------------------------------------
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 06f9c39..0830d1c 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
@@ -2207,7 +2207,7 @@ STAGE PLANS:
                         keys: _col0 (type: smallint)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                        Statistics: Num rows: 1140 Data size: 76368 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1156 Data size: 77440 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: smallint)
                           sort order: +
@@ -2216,7 +2216,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: 1140 Data size: 76368 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1156 Data size: 77440 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: double), _col6 (type: double), _col7 (type: bigint), _col8 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -2251,7 +2251,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: smallint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 1140 Data size: 76368 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1156 Data size: 77440 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: smallint), (UDFToInteger(_col0) % -75) (type: int), power(((_col1 - ((_col2 * _col2) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5) (type: double), (-1.389 / CAST( _col0 AS decimal(5,0))) (type: decimal(10,9)), _col4 (type: bigint), (UDFToDouble((UDFToInteger(_col0) % -75)) / UDFToDouble(_col4)) (type: double), (- (UDFToInteger(_col0) % -75)) (type: int), ((_col5 - ((_col6 * _col6) / _col7)) / _col7) (type: double), (- (- (UDFToInteger(_col0) % -75))) (type: int), _col8 (type: bigint), (_col8 - -89010L) (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -2260,7 +2260,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 9, 10, 16, 4, 18, 19, 17, 14, 8, 20]
                       selectExpressions: LongColModuloLongScalar(col 0:int, val -75)(children: col 0:smallint) -> 9:int, FuncPowerDoubleToDouble(col 11:double)(children: DoubleColDivideLongColumn(col 10:double, col 14:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 11:double)(children: DoubleColDivideLongColumn(col 10:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, col 2:double) -> 10:double) -> 11:double) -> 10:double, IfExprNullCondExpr(col 12:boolean, null, col 13:bigint)(children: LongColEqualLongScalar(col 3:bigint, val 1) -> 12:boolean, LongColSubtractLongScalar(col 3:bigint, val 1) -> 13:bigint) -> 14:bigint) -> 11:double) -> 10:double, DecimalScalarDivideDecimalColumn(val -1.389, col 15:decimal(5,0))(children: CastLongToDecimal(col 0:smallint) -> 15:decimal(5,0)) -> 16:decimal(10,9), DoubleColDivideDoubleColumn(col 11:double, col 17:double)(children: CastLongToDouble(col 14:int)(children: LongColModuloLongScalar(col 0:int, val -7
 5)(children: col 0:smallint) -> 14:int) -> 11:double, CastLongToDouble(col 4:bigint) -> 17:double) -> 18:double, LongColUnaryMinus(col 14:int)(children: LongColModuloLongScalar(col 0:int, val -75)(children: col 0:smallint) -> 14:int) -> 19:int, DoubleColDivideLongColumn(col 11:double, col 7:bigint)(children: DoubleColSubtractDoubleColumn(col 5:double, col 17:double)(children: DoubleColDivideLongColumn(col 11:double, col 7:bigint)(children: DoubleColMultiplyDoubleColumn(col 6:double, col 6:double) -> 11:double) -> 17:double) -> 11:double) -> 17:double, LongColUnaryMinus(col 20:int)(children: LongColUnaryMinus(col 14:int)(children: LongColModuloLongScalar(col 0:int, val -75)(children: col 0:smallint) -> 14:int) -> 20:int) -> 14:int, LongColSubtractLongScalar(col 8:bigint, val -89010) -> 20:bigint
-                  Statistics: Num rows: 1140 Data size: 199488 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1156 Data size: 202288 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: double), _col3 (type: decimal(10,9)), _col4 (type: bigint), _col5 (type: double), _col6 (type: int), _col7 (type: double), _col8 (type: int), _col9 (type: bigint), _col10 (type: bigint)
                     sort order: +++++++++++
@@ -2268,7 +2268,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: 1140 Data size: 199488 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1156 Data size: 202288 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.1
         Reducer 3 
             Execution mode: vectorized, llap
@@ -2286,7 +2286,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]
-                Statistics: Num rows: 1140 Data size: 199488 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1156 Data size: 202288 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Limit Vectorization:
@@ -2484,7 +2484,7 @@ STAGE PLANS:
                         keys: _col0 (type: double)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 1135 Data size: 52192 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1127 Data size: 51824 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: double)
                           sort order: +
@@ -2493,7 +2493,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: 1135 Data size: 52192 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1127 Data size: 51824 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
@@ -2528,7 +2528,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1135 Data size: 52192 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1127 Data size: 51824 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)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END)) + -5638.15D) (type: double), ((- ((_col1 - ((_col2 * _col2) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END)) * ((2563.58D * ((_col1 - ((_col2 * _col2) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END)) + -5638.15D)) (type: double), _col5 (type: double), ((_col1 - ((_col2 * _col2) / _col3)) / _col3) (type: double), (_col0 - (- ((_col1 - ((_col2 * _col2) / _col3)) / CASE WHEN ((_col3 = 1L)) THE
 N (null) ELSE ((_col3 - 1)) END))) (type: double), power(((_col1 - ((_col2 * _col2) / _col3)) / _col3), 0.5) (type: double), (_col0 + ((_col1 - ((_col2 * _col2) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END)) (type: double), (_col0 * 762.0D) (type: double), _col2 (type: double), (-863.257D % (_col0 * 762.0D)) (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -2537,7 +2537,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 7, 6, 11, 4, 17, 20, 5, 23, 26, 14, 29, 30, 2, 34]
                       selectExpressions: DoubleColDivideLongColumn(col 6:double, col 10: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) -> 6:double, IfExprNullCondExpr(col 8:boolean, null, col 9:bigint)(children: LongColEqualLongScalar(col 3:bigint, val 1) -> 8:boolean, LongColSubtractLongScalar(col 3:bigint, val 1) -> 9:bigint) -> 10:bigint) -> 7:double, DoubleScalarMultiplyDoubleColumn(val 2563.58, col 11:double)(children: DoubleColDivideLongColumn(col 6:double, col 13:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 11:double)(children: DoubleColDivideLongColumn(col 6:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, col 2:double) -> 6:double) -> 11:double) -> 6:double, IfExprNullCondExpr(col 10:boolean, null, col 12:bigint)(children: LongColEqualLongScal
 ar(col 3:bigint, val 1) -> 10:boolean, LongColSubtractLongScalar(col 3:bigint, val 1) -> 12:bigint) -> 13:bigint) -> 11:double) -> 6:double, DoubleColUnaryMinus(col 14:double)(children: DoubleColDivideLongColumn(col 11:double, col 16:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 14:double)(children: DoubleColDivideLongColumn(col 11:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, col 2:double) -> 11:double) -> 14:double) -> 11:double, IfExprNullCondExpr(col 13:boolean, null, col 15:bigint)(children: LongColEqualLongScalar(col 3:bigint, val 1) -> 13:boolean, LongColSubtractLongScalar(col 3:bigint, val 1) -> 15:bigint) -> 16:bigint) -> 14:double) -> 11:double, DoubleColAddDoubleScalar(col 14:double, val -5638.15)(children: DoubleScalarMultiplyDoubleColumn(val 2563.58, col 17:double)(children: DoubleColDivideLongColumn(col 14:double, col 19:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 17:double)(children: DoubleColDivi
 deLongColumn(col 14:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, col 2:double) -> 14:double) -> 17:double) -> 14:double, IfExprNullCondExpr(col 16:boolean, null, col 18:bigint)(children: LongColEqualLongScalar(col 3:bigint, val 1) -> 16:boolean, LongColSubtractLongScalar(col 3:bigint, val 1) -> 18:bigint) -> 19:bigint) -> 17:double) -> 14:double) -> 17:double, DoubleColMultiplyDoubleColumn(col 14:double, col 23:double)(children: DoubleColUnaryMinus(col 20:double)(children: DoubleColDivideLongColumn(col 14:double, col 22:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 20:double)(children: DoubleColDivideLongColumn(col 14:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, col 2:double) -> 14:double) -> 20:double) -> 14:double, IfExprNullCondExpr(col 19:boolean, null, col 21:bigint)(children: LongColEqualLongScalar(col 3:bigint, val 1) -> 19:boolean, LongColSubtractLongScalar(col 3:bigint, val 1) -> 21:bigint) ->
  22:bigint) -> 20:double) -> 14:double, DoubleColAddDoubleScalar(col 20:double, val -5638.15)(children: DoubleScalarMultiplyDoubleColumn(val 2563.58, col 23:double)(children: DoubleColDivideLongColumn(col 20:double, col 25:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 23:double)(children: DoubleColDivideLongColumn(col 20:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, col 2:double) -> 20:double) -> 23:double) -> 20:double, IfExprNullCondExpr(col 22:boolean, null, col 24:bigint)(children: LongColEqualLongScalar(col 3:bigint, val 1) -> 22:boolean, LongColSubtractLongScalar(col 3:bigint, val 1) -> 24:bigint) -> 25:bigint) -> 23:double) -> 20:double) -> 23:double) -> 20:double, DoubleColDivideLongColumn(col 14:double, col 3:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 23:double)(children: DoubleColDivideLongColumn(col 14:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, col 2:double) -> 14:dou
 ble) -> 23:double) -> 14:double) -> 23:double, DoubleColSubtractDoubleColumn(col 0:double, col 14:double)(children: DoubleColUnaryMinus(col 26:double)(children: DoubleColDivideLongColumn(col 14:double, col 28:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 26:double)(children: DoubleColDivideLongColumn(col 14:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, col 2:double) -> 14:double) -> 26:double) -> 14:double, IfExprNullCondExpr(col 25:boolean, null, col 27:bigint)(children: LongColEqualLongScalar(col 3:bigint, val 1) -> 25:boolean, LongColSubtractLongScalar(col 3:bigint, val 1) -> 27:bigint) -> 28:bigint) -> 26:double) -> 14:double) -> 26:double, FuncPowerDoubleToDouble(col 29:double)(children: DoubleColDivideLongColumn(col 14:double, col 3:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 29:double)(children: DoubleColDivideLongColumn(col 14:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, co
 l 2:double) -> 14:double) -> 29:double) -> 14:double) -> 29:double) -> 14:double, DoubleColAddDoubleColumn(col 0:double, col 30:double)(children: DoubleColDivideLongColumn(col 29:double, col 32:bigint)(children: DoubleColSubtractDoubleColumn(col 1:double, col 30:double)(children: DoubleColDivideLongColumn(col 29:double, col 3:bigint)(children: DoubleColMultiplyDoubleColumn(col 2:double, col 2:double) -> 29:double) -> 30:double) -> 29:double, IfExprNullCondExpr(col 28:boolean, null, col 31:bigint)(children: LongColEqualLongScalar(col 3:bigint, val 1) -> 28:boolean, LongColSubtractLongScalar(col 3:bigint, val 1) -> 31:bigint) -> 32:bigint) -> 30:double) -> 29:double, DoubleColMultiplyDoubleScalar(col 0:double, val 762.0) -> 30:double, DoubleScalarModuloDoubleColumn(val -863.257, col 33:double)(children: DoubleColMultiplyDoubleScalar(col 0:double, val 762.0) -> 33:double) -> 34:double
-                  Statistics: Num rows: 1135 Data size: 142992 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1127 Data size: 141984 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: double)
                     sort order: +
@@ -2545,7 +2545,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: 1135 Data size: 142992 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1127 Data size: 141984 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
@@ -2563,13 +2563,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: 1135 Data size: 142992 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1127 Data size: 141984 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 1135 Data size: 142992 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1127 Data size: 141984 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorized_context.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_context.q.out b/ql/src/test/results/clientpositive/llap/vectorized_context.q.out
index bc1a203..3edc12b 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_context.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_context.q.out
@@ -133,7 +133,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col2
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 6126 Data size: 49056 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6062 Data size: 48288 Basic stats: COMPLETE Column stats: COMPLETE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
@@ -143,17 +143,17 @@ STAGE PLANS:
                           outputColumnNames: _col2, _col5
                           input vertices:
                             1 Map 3
-                          Statistics: Num rows: 6177 Data size: 626386 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 6049 Data size: 612818 Basic stats: COMPLETE Column stats: COMPLETE
                           Select Operator
                             expressions: _col5 (type: string), _col2 (type: double)
                             outputColumnNames: _col0, _col1
-                            Statistics: Num rows: 6177 Data size: 626386 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 6049 Data size: 612818 Basic stats: COMPLETE Column stats: COMPLETE
                             Limit
                               Number of rows: 100
-                              Statistics: Num rows: 100 Data size: 10208 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 100 Data size: 10200 Basic stats: COMPLETE Column stats: COMPLETE
                               File Output Operator
                                 compressed: false
-                                Statistics: Num rows: 100 Data size: 10208 Basic stats: COMPLETE Column stats: COMPLETE
+                                Statistics: Num rows: 100 Data size: 10200 Basic stats: COMPLETE Column stats: COMPLETE
                                 table:
                                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorized_distinct_gby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_distinct_gby.q.out b/ql/src/test/results/clientpositive/llap/vectorized_distinct_gby.q.out
index 05bf12c..de3c6e6 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_distinct_gby.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_distinct_gby.q.out
@@ -209,7 +209,7 @@ STAGE PLANS:
                       keys: cint (type: int)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 6027 Data size: 18000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6105 Data size: 18232 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
@@ -220,7 +220,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
                             valueColumnNums: []
-                        Statistics: Num rows: 6027 Data size: 18000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 6105 Data size: 18232 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -265,7 +265,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 6027 Data size: 18000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 6105 Data size: 18232 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), UDFToDouble(_col0) (type: double), (UDFToDouble(_col0) * UDFToDouble(_col0)) (type: double)
                   outputColumnNames: _col0, _col1, _col2
@@ -274,7 +274,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 1, 4]
                       selectExpressions: CastLongToDouble(col 0:int) -> 1:double, DoubleColMultiplyDoubleColumn(col 2:double, col 3:double)(children: CastLongToDouble(col 0:int) -> 2:double, CastLongToDouble(col 0:int) -> 3:double) -> 4:double
-                  Statistics: Num rows: 6027 Data size: 18000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6105 Data size: 18232 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), count(_col0), sum(_col2), sum(_col1)
                     Group By Vectorization:

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out
index 7bacad8..b30fbf3 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out
@@ -58,7 +58,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 18702 Data size: 131024 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 18464 Data size: 129120 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: int), _col1 (type: int), (_col0 + _col1) (type: int)
                           outputColumnNames: _col0, _col1, _col2
@@ -67,7 +67,7 @@ STAGE PLANS:
                               native: true
                               projectedOutputColumnNums: [2, 2, 13]
                               selectExpressions: LongColAddLongColumn(col 2:int, col 2:int) -> 13:int
-                          Statistics: Num rows: 18702 Data size: 131024 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 18464 Data size: 129120 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
                             aggregations: count(_col0), max(_col1), min(_col0), sum(_col2), count(_col2)
                             Group By Vectorization:

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorized_nested_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_nested_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vectorized_nested_mapjoin.q.out
index 8e3323b..d27a123 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_nested_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_nested_mapjoin.q.out
@@ -40,11 +40,11 @@ STAGE PLANS:
                         outputColumnNames: _col2, _col3
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 661228 Data size: 7913928 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 640986 Data size: 7671024 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col2 (type: smallint), _col3 (type: double)
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 661228 Data size: 7913928 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 640986 Data size: 7671024 Basic stats: COMPLETE Column stats: COMPLETE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -54,7 +54,7 @@ STAGE PLANS:
                             outputColumnNames: _col1
                             input vertices:
                               1 Map 4
-                            Statistics: Num rows: 1452959 Data size: 11609800 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 1389838 Data size: 11104832 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
                               aggregations: sum(_col1)
                               mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorized_parquet.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_parquet.q.out b/ql/src/test/results/clientpositive/llap/vectorized_parquet.q.out
index ab30d6e..59adda0 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_parquet.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_parquet.q.out
@@ -151,12 +151,12 @@ STAGE PLANS:
                       keys: _col0 (type: tinyint)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                      Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: tinyint)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: tinyint)
-                        Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: double), _col5 (type: bigint), _col6 (type: double), _col7 (type: double), _col8 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs (cache only)
@@ -183,14 +183,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 128 Data size: 7556 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 7732 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), (_col4 / _col5) (type: double), power(((_col6 - ((_col7 * _col7) / _col8)) / _col8), 0.5) (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 128 Data size: 4484 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 4588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 128 Data size: 4484 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 4588 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out b/ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out
index 776e86c..690f0ae 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out
@@ -118,11 +118,11 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 18702 Data size: 131024 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 18464 Data size: 129120 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: int), (_col0 + _col1) (type: int)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 18702 Data size: 131024 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 18464 Data size: 129120 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count(_col0), max(_col1), min(_col0), sum(_col2), count(_col2)
                     mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out b/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
index 74f52f5..a696961 100644
--- a/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
@@ -536,24 +536,24 @@ STAGE PLANS:
   Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_1:src 
+        $hdt$_1:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_1:src 
+        $hdt$_1:src1 
           TableScan
-            alias: src
+            alias: src1
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: value is not null (type: boolean)
+              predicate: key is not null (type: boolean)
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: value (type: string)
+                expressions: key (type: string)
                 outputColumnNames: _col0
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 HashTable Sink Operator
                   keys:
-                    0 _col1 (type: string)
+                    0 _col0 (type: string)
                     1 _col0 (type: string)
 
   Stage: Stage-7
@@ -573,9 +573,9 @@ STAGE PLANS:
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col1 (type: string)
+                    0 _col0 (type: string)
                     1 _col0 (type: string)
-                  outputColumnNames: _col0, _col2
+                  outputColumnNames: _col1, _col2
                   Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
@@ -590,24 +590,24 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_2:src1 
+        $hdt$_2:src 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_2:src1 
+        $hdt$_2:src 
           TableScan
-            alias: src1
+            alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
+              predicate: value is not null (type: boolean)
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: key (type: string)
+                expressions: value (type: string)
                 outputColumnNames: _col0
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 HashTable Sink Operator
                   keys:
-                    0 _col0 (type: string)
+                    0 _col1 (type: string)
                     1 _col0 (type: string)
 
   Stage: Stage-3
@@ -618,7 +618,7 @@ STAGE PLANS:
               condition map:
                    Inner Join 0 to 1
               keys:
-                0 _col0 (type: string)
+                0 _col1 (type: string)
                 1 _col0 (type: string)
               outputColumnNames: _col2
               Statistics: Num rows: 2420 Data size: 25709 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
index cb1c69f..8b47d7f 100644
--- a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
@@ -4335,38 +4335,38 @@ STAGE PLANS:
         Map 2 
             Map Operator Tree:
                 TableScan
-                  alias: c
+                  alias: b
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Spark HashTable Sink Operator
                         keys:
-                          0 _col0 (type: int)
-                          1 _col0 (type: int)
+                          0 _col1 (type: string)
+                          1 _col1 (type: string)
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
         Map 3 
             Map Operator Tree:
                 TableScan
-                  alias: b
+                  alias: c
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: value is not null (type: boolean)
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Spark HashTable Sink Operator
                         keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
@@ -4391,9 +4391,9 @@ STAGE PLANS:
                         condition map:
                              Inner Join 0 to 1
                         keys:
-                          0 _col0 (type: int)
-                          1 _col0 (type: int)
-                        outputColumnNames: _col0, _col1
+                          0 _col1 (type: string)
+                          1 _col1 (type: string)
+                        outputColumnNames: _col0, _col2
                         input vertices:
                           1 Map 2
                         Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
@@ -4401,14 +4401,14 @@ STAGE PLANS:
                           condition map:
                                Inner Join 0 to 1
                           keys:
-                            0 _col1 (type: string)
-                            1 _col1 (type: string)
-                          outputColumnNames: _col0, _col3
+                            0 _col0 (type: int)
+                            1 _col0 (type: int)
+                          outputColumnNames: _col0, _col2
                           input vertices:
                             1 Map 3
                           Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                           Select Operator
-                            expressions: _col0 (type: int), _col3 (type: int)
+                            expressions: _col0 (type: int), _col2 (type: int)
                             outputColumnNames: _col0, _col1
                             Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                             File Output Operator
@@ -4445,38 +4445,38 @@ STAGE PLANS:
         Map 2 
             Map Operator Tree:
                 TableScan
-                  alias: c
+                  alias: b
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Spark HashTable Sink Operator
                         keys:
-                          0 _col0 (type: int)
-                          1 _col0 (type: int)
+                          0 _col1 (type: string)
+                          1 _col1 (type: string)
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
         Map 3 
             Map Operator Tree:
                 TableScan
-                  alias: b
+                  alias: c
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: value is not null (type: boolean)
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Spark HashTable Sink Operator
                         keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
@@ -4501,9 +4501,9 @@ STAGE PLANS:
                         condition map:
                              Inner Join 0 to 1
                         keys:
-                          0 _col0 (type: int)
-                          1 _col0 (type: int)
-                        outputColumnNames: _col0, _col1
+                          0 _col1 (type: string)
+                          1 _col1 (type: string)
+                        outputColumnNames: _col0, _col2
                         input vertices:
                           1 Map 2
                         Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
@@ -4511,14 +4511,14 @@ STAGE PLANS:
                           condition map:
                                Inner Join 0 to 1
                           keys:
-                            0 _col1 (type: string)
-                            1 _col1 (type: string)
-                          outputColumnNames: _col0, _col3
+                            0 _col0 (type: int)
+                            1 _col0 (type: int)
+                          outputColumnNames: _col0, _col2
                           input vertices:
                             1 Map 3
                           Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                           Select Operator
-                            expressions: _col0 (type: int), _col3 (type: int)
+                            expressions: _col0 (type: int), _col2 (type: int)
                             outputColumnNames: _col0, _col1
                             Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                             File Output Operator


[08/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_2.q.out b/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
index 361dc07..735915e 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
@@ -192,9 +192,9 @@ Stage-0
     Stage-1
       Reducer 3 llap
       File Output Operator [FS_16]
-        Select Operator [SEL_15] (rows=64 width=268)
+        Select Operator [SEL_15] (rows=63 width=268)
           Output:["_col0","_col1","_col2"]
-          Merge Join Operator [MERGEJOIN_26] (rows=64 width=268)
+          Merge Join Operator [MERGEJOIN_26] (rows=63 width=268)
             Conds:RS_12._col3=RS_35._col0(Inner),Output:["_col1","_col2","_col4"]
           <-Map 5 [SIMPLE_EDGE] vectorized, llap
             SHUFFLE [RS_35]
@@ -208,7 +208,7 @@ Stage-0
           <-Reducer 2 [SIMPLE_EDGE] llap
             SHUFFLE [RS_12]
               PartitionCols:_col3
-              Merge Join Operator [MERGEJOIN_25] (rows=40 width=266)
+              Merge Join Operator [MERGEJOIN_25] (rows=39 width=266)
                 Conds:RS_29._col0=RS_32._col0(Inner),Output:["_col1","_col2","_col3"]
               <-Map 1 [SIMPLE_EDGE] vectorized, llap
                 SHUFFLE [RS_29]
@@ -443,146 +443,141 @@ POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Union 2 (CONTAINS)
+Map 12 <- Union 10 (CONTAINS)
 Map 13 <- Union 14 (CONTAINS)
 Map 16 <- Union 14 (CONTAINS)
-Map 8 <- Union 2 (CONTAINS)
-Reducer 10 <- Map 9 (SIMPLE_EDGE), Reducer 15 (SIMPLE_EDGE)
-Reducer 11 <- Map 17 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Map 9 <- Union 10 (CONTAINS)
+Reducer 11 <- Union 10 (SIMPLE_EDGE)
 Reducer 15 <- Union 14 (SIMPLE_EDGE)
-Reducer 3 <- Union 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 12 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 7 <- Union 6 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 11 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 5 <- Union 4 (SIMPLE_EDGE)
+Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 15 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE), Union 4 (CONTAINS)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 7 vectorized, llap
-      File Output Operator [FS_101]
-        Group By Operator [GBY_100] (rows=28 width=177)
+      Reducer 5 vectorized, llap
+      File Output Operator [FS_96]
+        Group By Operator [GBY_95] (rows=33 width=177)
           Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-        <-Union 6 [SIMPLE_EDGE]
-          <-Reducer 11 [CONTAINS] llap
+        <-Union 4 [SIMPLE_EDGE]
+          <-Reducer 3 [CONTAINS] llap
             Reduce Output Operator [RS_56]
               PartitionCols:_col0, _col1
-              Group By Operator [GBY_55] (rows=28 width=177)
+              Group By Operator [GBY_55] (rows=33 width=177)
                 Output:["_col0","_col1"],keys:_col0, _col1
-                Select Operator [SEL_51] (rows=33 width=177)
+                Select Operator [SEL_25] (rows=33 width=177)
                   Output:["_col0","_col1"]
-                  Merge Join Operator [MERGEJOIN_85] (rows=33 width=177)
-                    Conds:RS_48._col2=RS_111._col0(Inner),Output:["_col1","_col2"]
-                  <-Map 17 [SIMPLE_EDGE] vectorized, llap
-                    SHUFFLE [RS_111]
-                      PartitionCols:_col0
-                      Select Operator [SEL_110] (rows=500 width=87)
-                        Output:["_col0"]
-                        Filter Operator [FIL_109] (rows=500 width=87)
-                          predicate:key is not null
-                          TableScan [TS_42] (rows=500 width=87)
-                            default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
-                  <-Reducer 10 [SIMPLE_EDGE] llap
-                    SHUFFLE [RS_48]
+                  Merge Join Operator [MERGEJOIN_82] (rows=33 width=177)
+                    Conds:RS_22._col2=RS_94._col1(Inner),Output:["_col1","_col4"]
+                  <-Reducer 11 [SIMPLE_EDGE] vectorized, llap
+                    SHUFFLE [RS_94]
+                      PartitionCols:_col1
+                      Select Operator [SEL_93] (rows=262 width=178)
+                        Output:["_col1"]
+                        Group By Operator [GBY_92] (rows=262 width=178)
+                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                        <-Union 10 [SIMPLE_EDGE]
+                          <-Map 12 [CONTAINS] vectorized, llap
+                            Reduce Output Operator [RS_107]
+                              PartitionCols:_col0, _col1
+                              Group By Operator [GBY_106] (rows=262 width=178)
+                                Output:["_col0","_col1"],keys:_col1, _col0
+                                Select Operator [SEL_105] (rows=500 width=178)
+                                  Output:["_col0","_col1"]
+                                  Filter Operator [FIL_104] (rows=500 width=178)
+                                    predicate:value is not null
+                                    TableScan [TS_9] (rows=500 width=178)
+                                      Output:["key","value"]
+                          <-Map 9 [CONTAINS] vectorized, llap
+                            Reduce Output Operator [RS_103]
+                              PartitionCols:_col0, _col1
+                              Group By Operator [GBY_102] (rows=262 width=178)
+                                Output:["_col0","_col1"],keys:_col1, _col0
+                                Select Operator [SEL_101] (rows=25 width=175)
+                                  Output:["_col0","_col1"]
+                                  Filter Operator [FIL_100] (rows=25 width=175)
+                                    predicate:value is not null
+                                    TableScan [TS_6] (rows=25 width=175)
+                                      Output:["key","value"]
+                  <-Reducer 2 [SIMPLE_EDGE] llap
+                    SHUFFLE [RS_22]
                       PartitionCols:_col2
-                      Merge Join Operator [MERGEJOIN_84] (rows=21 width=177)
-                        Conds:RS_108._col1=RS_96._col1(Inner),Output:["_col1","_col2"]
-                      <-Map 9 [SIMPLE_EDGE] vectorized, llap
-                        SHUFFLE [RS_96]
-                          PartitionCols:_col1
-                          Select Operator [SEL_94] (rows=25 width=175)
+                      Merge Join Operator [MERGEJOIN_80] (rows=39 width=175)
+                        Conds:RS_86._col0=RS_90._col0(Inner),Output:["_col1","_col2"]
+                      <-Map 1 [SIMPLE_EDGE] vectorized, llap
+                        SHUFFLE [RS_86]
+                          PartitionCols:_col0
+                          Select Operator [SEL_85] (rows=500 width=87)
+                            Output:["_col0"]
+                            Filter Operator [FIL_84] (rows=500 width=87)
+                              predicate:key is not null
+                              TableScan [TS_0] (rows=500 width=87)
+                                default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
+                      <-Map 8 [SIMPLE_EDGE] vectorized, llap
+                        SHUFFLE [RS_90]
+                          PartitionCols:_col0
+                          Select Operator [SEL_89] (rows=25 width=175)
                             Output:["_col0","_col1"]
-                            Filter Operator [FIL_93] (rows=25 width=175)
+                            Filter Operator [FIL_88] (rows=25 width=175)
                               predicate:(key is not null and value is not null)
-                              TableScan [TS_13] (rows=25 width=175)
+                              TableScan [TS_3] (rows=25 width=175)
                                 default@src1,x,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                      <-Reducer 15 [SIMPLE_EDGE] vectorized, llap
-                        SHUFFLE [RS_108]
-                          PartitionCols:_col1
-                          Select Operator [SEL_107] (rows=262 width=178)
-                            Output:["_col1"]
-                            Group By Operator [GBY_106] (rows=262 width=178)
-                              Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                            <-Union 14 [SIMPLE_EDGE]
-                              <-Map 13 [CONTAINS] vectorized, llap
-                                Reduce Output Operator [RS_115]
-                                  PartitionCols:_col0, _col1
-                                  Group By Operator [GBY_114] (rows=262 width=178)
-                                    Output:["_col0","_col1"],keys:_col1, _col0
-                                    Select Operator [SEL_113] (rows=25 width=175)
-                                      Output:["_col0","_col1"]
-                                      Filter Operator [FIL_112] (rows=25 width=175)
-                                        predicate:value is not null
-                                        TableScan [TS_26] (rows=25 width=175)
-                                          Output:["key","value"]
-                              <-Map 16 [CONTAINS] vectorized, llap
-                                Reduce Output Operator [RS_119]
-                                  PartitionCols:_col0, _col1
-                                  Group By Operator [GBY_118] (rows=262 width=178)
-                                    Output:["_col0","_col1"],keys:_col1, _col0
-                                    Select Operator [SEL_117] (rows=500 width=178)
-                                      Output:["_col0","_col1"]
-                                      Filter Operator [FIL_116] (rows=500 width=178)
-                                        predicate:value is not null
-                                        TableScan [TS_29] (rows=500 width=178)
-                                          Output:["key","value"]
-          <-Reducer 5 [CONTAINS] llap
+          <-Reducer 7 [CONTAINS] llap
             Reduce Output Operator [RS_56]
               PartitionCols:_col0, _col1
-              Group By Operator [GBY_55] (rows=28 width=177)
+              Group By Operator [GBY_55] (rows=33 width=177)
                 Output:["_col0","_col1"],keys:_col0, _col1
-                Select Operator [SEL_25] (rows=33 width=177)
+                Select Operator [SEL_51] (rows=33 width=177)
                   Output:["_col0","_col1"]
                   Merge Join Operator [MERGEJOIN_83] (rows=33 width=177)
-                    Conds:RS_22._col2=RS_99._col0(Inner),Output:["_col1","_col2"]
-                  <-Map 12 [SIMPLE_EDGE] vectorized, llap
+                    Conds:RS_48._col2=RS_99._col1(Inner),Output:["_col1","_col4"]
+                  <-Reducer 15 [SIMPLE_EDGE] vectorized, llap
                     SHUFFLE [RS_99]
-                      PartitionCols:_col0
-                      Select Operator [SEL_98] (rows=500 width=87)
-                        Output:["_col0"]
-                        Filter Operator [FIL_97] (rows=500 width=87)
-                          predicate:key is not null
-                          TableScan [TS_16] (rows=500 width=87)
-                            default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
-                  <-Reducer 4 [SIMPLE_EDGE] llap
-                    SHUFFLE [RS_22]
+                      PartitionCols:_col1
+                      Select Operator [SEL_98] (rows=262 width=178)
+                        Output:["_col1"]
+                        Group By Operator [GBY_97] (rows=262 width=178)
+                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                        <-Union 14 [SIMPLE_EDGE]
+                          <-Map 13 [CONTAINS] vectorized, llap
+                            Reduce Output Operator [RS_111]
+                              PartitionCols:_col0, _col1
+                              Group By Operator [GBY_110] (rows=262 width=178)
+                                Output:["_col0","_col1"],keys:_col1, _col0
+                                Select Operator [SEL_109] (rows=25 width=175)
+                                  Output:["_col0","_col1"]
+                                  Filter Operator [FIL_108] (rows=25 width=175)
+                                    predicate:value is not null
+                                    TableScan [TS_32] (rows=25 width=175)
+                                      Output:["key","value"]
+                          <-Map 16 [CONTAINS] vectorized, llap
+                            Reduce Output Operator [RS_115]
+                              PartitionCols:_col0, _col1
+                              Group By Operator [GBY_114] (rows=262 width=178)
+                                Output:["_col0","_col1"],keys:_col1, _col0
+                                Select Operator [SEL_113] (rows=500 width=178)
+                                  Output:["_col0","_col1"]
+                                  Filter Operator [FIL_112] (rows=500 width=178)
+                                    predicate:value is not null
+                                    TableScan [TS_35] (rows=500 width=178)
+                                      Output:["key","value"]
+                  <-Reducer 6 [SIMPLE_EDGE] llap
+                    SHUFFLE [RS_48]
                       PartitionCols:_col2
-                      Merge Join Operator [MERGEJOIN_82] (rows=21 width=177)
-                        Conds:RS_92._col1=RS_95._col1(Inner),Output:["_col1","_col2"]
-                      <-Map 9 [SIMPLE_EDGE] vectorized, llap
-                        SHUFFLE [RS_95]
-                          PartitionCols:_col1
-                           Please refer to the previous Select Operator [SEL_94]
-                      <-Reducer 3 [SIMPLE_EDGE] vectorized, llap
-                        SHUFFLE [RS_92]
-                          PartitionCols:_col1
-                          Select Operator [SEL_91] (rows=262 width=178)
-                            Output:["_col1"]
-                            Group By Operator [GBY_90] (rows=262 width=178)
-                              Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                            <-Union 2 [SIMPLE_EDGE]
-                              <-Map 1 [CONTAINS] vectorized, llap
-                                Reduce Output Operator [RS_89]
-                                  PartitionCols:_col0, _col1
-                                  Group By Operator [GBY_88] (rows=262 width=178)
-                                    Output:["_col0","_col1"],keys:_col1, _col0
-                                    Select Operator [SEL_87] (rows=25 width=175)
-                                      Output:["_col0","_col1"]
-                                      Filter Operator [FIL_86] (rows=25 width=175)
-                                        predicate:value is not null
-                                        TableScan [TS_0] (rows=25 width=175)
-                                          Output:["key","value"]
-                              <-Map 8 [CONTAINS] vectorized, llap
-                                Reduce Output Operator [RS_105]
-                                  PartitionCols:_col0, _col1
-                                  Group By Operator [GBY_104] (rows=262 width=178)
-                                    Output:["_col0","_col1"],keys:_col1, _col0
-                                    Select Operator [SEL_103] (rows=500 width=178)
-                                      Output:["_col0","_col1"]
-                                      Filter Operator [FIL_102] (rows=500 width=178)
-                                        predicate:value is not null
-                                        TableScan [TS_3] (rows=500 width=178)
-                                          Output:["key","value"]
+                      Merge Join Operator [MERGEJOIN_81] (rows=39 width=175)
+                        Conds:RS_87._col0=RS_91._col0(Inner),Output:["_col1","_col2"]
+                      <-Map 1 [SIMPLE_EDGE] vectorized, llap
+                        SHUFFLE [RS_87]
+                          PartitionCols:_col0
+                           Please refer to the previous Select Operator [SEL_85]
+                      <-Map 8 [SIMPLE_EDGE] vectorized, llap
+                        SHUFFLE [RS_91]
+                          PartitionCols:_col0
+                           Please refer to the previous Select Operator [SEL_89]
 
 PREHOOK: query: explain
 SELECT x.key, y.value
@@ -613,281 +608,266 @@ POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Union 2 (CONTAINS)
-Map 10 <- Union 2 (CONTAINS)
-Map 17 <- Union 18 (CONTAINS)
-Map 22 <- Union 18 (CONTAINS)
-Map 23 <- Union 20 (CONTAINS)
-Map 25 <- Union 26 (CONTAINS)
-Map 32 <- Union 26 (CONTAINS)
-Map 33 <- Union 28 (CONTAINS)
-Map 34 <- Union 30 (CONTAINS)
-Reducer 12 <- Map 11 (SIMPLE_EDGE), Reducer 21 (SIMPLE_EDGE)
-Reducer 13 <- Map 24 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 14 <- Map 11 (SIMPLE_EDGE), Reducer 31 (SIMPLE_EDGE)
-Reducer 15 <- Map 16 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE), Union 8 (CONTAINS)
-Reducer 19 <- Union 18 (SIMPLE_EDGE), Union 20 (CONTAINS)
-Reducer 21 <- Union 20 (SIMPLE_EDGE)
+Map 12 <- Union 13 (CONTAINS)
+Map 15 <- Union 13 (CONTAINS)
+Map 16 <- Union 17 (CONTAINS)
+Map 21 <- Union 17 (CONTAINS)
+Map 22 <- Union 19 (CONTAINS)
+Map 23 <- Union 24 (CONTAINS)
+Map 30 <- Union 24 (CONTAINS)
+Map 31 <- Union 26 (CONTAINS)
+Map 32 <- Union 28 (CONTAINS)
+Reducer 10 <- Reducer 20 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 14 <- Union 13 (SIMPLE_EDGE)
+Reducer 18 <- Union 17 (SIMPLE_EDGE), Union 19 (CONTAINS)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
+Reducer 20 <- Union 19 (SIMPLE_EDGE)
+Reducer 25 <- Union 24 (SIMPLE_EDGE), Union 26 (CONTAINS)
 Reducer 27 <- Union 26 (SIMPLE_EDGE), Union 28 (CONTAINS)
-Reducer 29 <- Union 28 (SIMPLE_EDGE), Union 30 (CONTAINS)
-Reducer 3 <- Union 2 (SIMPLE_EDGE)
-Reducer 31 <- Union 30 (SIMPLE_EDGE)
-Reducer 4 <- Map 11 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 16 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 7 <- Union 6 (SIMPLE_EDGE), Union 8 (CONTAINS)
-Reducer 9 <- Union 8 (SIMPLE_EDGE)
+Reducer 29 <- Union 28 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 14 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 5 <- Union 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 7 <- Union 6 (SIMPLE_EDGE)
+Reducer 8 <- Reducer 2 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 9 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 9 vectorized, llap
-      File Output Operator [FS_191]
-        Group By Operator [GBY_190] (rows=48 width=177)
+      Reducer 7 vectorized, llap
+      File Output Operator [FS_180]
+        Group By Operator [GBY_179] (rows=47 width=177)
           Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-        <-Union 8 [SIMPLE_EDGE]
-          <-Reducer 15 [CONTAINS] llap
-            Reduce Output Operator [RS_119]
-              PartitionCols:_col0, _col1
-              Group By Operator [GBY_118] (rows=48 width=177)
-                Output:["_col0","_col1"],keys:_col0, _col1
-                Select Operator [SEL_114] (rows=56 width=177)
-                  Output:["_col0","_col1"]
-                  Merge Join Operator [MERGEJOIN_170] (rows=56 width=177)
-                    Conds:RS_111._col2=RS_186._col0(Inner),Output:["_col2","_col5"]
-                  <-Map 16 [SIMPLE_EDGE] vectorized, llap
-                    SHUFFLE [RS_186]
-                      PartitionCols:_col0
-                      Select Operator [SEL_184] (rows=500 width=178)
-                        Output:["_col0","_col1"]
-                        Filter Operator [FIL_183] (rows=500 width=178)
-                          predicate:key is not null
-                          TableScan [TS_16] (rows=500 width=178)
-                            default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                  <-Reducer 14 [SIMPLE_EDGE] llap
-                    SHUFFLE [RS_111]
-                      PartitionCols:_col2
-                      Merge Join Operator [MERGEJOIN_169] (rows=35 width=86)
-                        Conds:RS_204._col1=RS_182._col1(Inner),Output:["_col2"]
-                      <-Map 11 [SIMPLE_EDGE] vectorized, llap
-                        SHUFFLE [RS_182]
-                          PartitionCols:_col1
-                          Select Operator [SEL_179] (rows=25 width=175)
-                            Output:["_col0","_col1"]
-                            Filter Operator [FIL_178] (rows=25 width=175)
-                              predicate:(key is not null and value is not null)
-                              TableScan [TS_13] (rows=25 width=175)
-                                default@src1,x,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                      <-Reducer 31 [SIMPLE_EDGE] vectorized, llap
-                        SHUFFLE [RS_204]
-                          PartitionCols:_col1
-                          Select Operator [SEL_203] (rows=440 width=178)
-                            Output:["_col1"]
-                            Group By Operator [GBY_202] (rows=440 width=178)
-                              Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                            <-Union 30 [SIMPLE_EDGE]
-                              <-Map 34 [CONTAINS] vectorized, llap
-                                Reduce Output Operator [RS_244]
-                                  PartitionCols:_col0, _col1
-                                  Group By Operator [GBY_243] (rows=440 width=178)
-                                    Output:["_col0","_col1"],keys:_col1, _col0
-                                    Select Operator [SEL_242] (rows=500 width=178)
-                                      Output:["_col0","_col1"]
-                                      Filter Operator [FIL_241] (rows=500 width=178)
-                                        predicate:value is not null
-                                        TableScan [TS_92] (rows=500 width=178)
-                                          Output:["key","value"]
-                              <-Reducer 29 [CONTAINS] vectorized, llap
-                                Reduce Output Operator [RS_232]
-                                  PartitionCols:_col0, _col1
-                                  Group By Operator [GBY_231] (rows=440 width=178)
-                                    Output:["_col0","_col1"],keys:_col1, _col0
-                                    Select Operator [SEL_230] (rows=381 width=178)
-                                      Output:["_col0","_col1"]
-                                      Group By Operator [GBY_229] (rows=381 width=178)
-                                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                      <-Union 28 [SIMPLE_EDGE]
-                                        <-Map 33 [CONTAINS] vectorized, llap
-                                          Reduce Output Operator [RS_240]
-                                            PartitionCols:_col0, _col1
-                                            Group By Operator [GBY_239] (rows=381 width=178)
-                                              Output:["_col0","_col1"],keys:_col1, _col0
-                                              Select Operator [SEL_238] (rows=500 width=178)
-                                                Output:["_col0","_col1"]
-                                                Filter Operator [FIL_237] (rows=500 width=178)
-                                                  predicate:value is not null
-                                                  TableScan [TS_82] (rows=500 width=178)
-                                                    Output:["key","value"]
-                                        <-Reducer 27 [CONTAINS] vectorized, llap
-                                          Reduce Output Operator [RS_228]
-                                            PartitionCols:_col0, _col1
-                                            Group By Operator [GBY_227] (rows=381 width=178)
-                                              Output:["_col0","_col1"],keys:_col1, _col0
-                                              Select Operator [SEL_226] (rows=262 width=178)
-                                                Output:["_col0","_col1"]
-                                                Group By Operator [GBY_225] (rows=262 width=178)
-                                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                                <-Union 26 [SIMPLE_EDGE]
-                                                  <-Map 25 [CONTAINS] vectorized, llap
-                                                    Reduce Output Operator [RS_224]
-                                                      PartitionCols:_col0, _col1
-                                                      Group By Operator [GBY_223] (rows=262 width=178)
-                                                        Output:["_col0","_col1"],keys:_col1, _col0
-                                                        Select Operator [SEL_222] (rows=25 width=175)
-                                                          Output:["_col0","_col1"]
-                                                          Filter Operator [FIL_221] (rows=25 width=175)
-                                                            predicate:value is not null
-                                                            TableScan [TS_69] (rows=25 width=175)
-                                                              Output:["key","value"]
-                                                  <-Map 32 [CONTAINS] vectorized, llap
-                                                    Reduce Output Operator [RS_236]
-                                                      PartitionCols:_col0, _col1
-                                                      Group By Operator [GBY_235] (rows=262 width=178)
-                                                        Output:["_col0","_col1"],keys:_col1, _col0
-                                                        Select Operator [SEL_234] (rows=500 width=178)
-                                                          Output:["_col0","_col1"]
-                                                          Filter Operator [FIL_233] (rows=500 width=178)
-                                                            predicate:value is not null
-                                                            TableScan [TS_72] (rows=500 width=178)
-                                                              Output:["key","value"]
-          <-Reducer 7 [CONTAINS] vectorized, llap
-            Reduce Output Operator [RS_189]
+        <-Union 6 [SIMPLE_EDGE]
+          <-Reducer 5 [CONTAINS] vectorized, llap
+            Reduce Output Operator [RS_178]
               PartitionCols:_col0, _col1
-              Group By Operator [GBY_188] (rows=48 width=177)
+              Group By Operator [GBY_177] (rows=47 width=177)
                 Output:["_col0","_col1"],keys:_col0, _col1
-                Group By Operator [GBY_187] (rows=40 width=177)
+                Group By Operator [GBY_176] (rows=40 width=177)
                   Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                <-Union 6 [SIMPLE_EDGE]
-                  <-Reducer 13 [CONTAINS] llap
+                <-Union 4 [SIMPLE_EDGE]
+                  <-Reducer 10 [CONTAINS] llap
                     Reduce Output Operator [RS_66]
                       PartitionCols:_col0, _col1
                       Group By Operator [GBY_65] (rows=40 width=177)
                         Output:["_col0","_col1"],keys:_col0, _col1
                         Select Operator [SEL_61] (rows=48 width=177)
                           Output:["_col0","_col1"]
-                          Merge Join Operator [MERGEJOIN_168] (rows=48 width=177)
-                            Conds:RS_58._col2=RS_201._col0(Inner),Output:["_col2","_col5"]
-                          <-Map 24 [SIMPLE_EDGE] vectorized, llap
-                            SHUFFLE [RS_201]
-                              PartitionCols:_col0
-                              Select Operator [SEL_200] (rows=500 width=178)
-                                Output:["_col0","_col1"]
-                                Filter Operator [FIL_199] (rows=500 width=178)
-                                  predicate:key is not null
-                                  TableScan [TS_52] (rows=500 width=178)
-                                    default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                          <-Reducer 12 [SIMPLE_EDGE] llap
+                          Merge Join Operator [MERGEJOIN_163] (rows=48 width=177)
+                            Conds:RS_58._col3=RS_186._col1(Inner),Output:["_col1","_col2"]
+                          <-Reducer 20 [SIMPLE_EDGE] vectorized, llap
+                            SHUFFLE [RS_186]
+                              PartitionCols:_col1
+                              Select Operator [SEL_185] (rows=381 width=178)
+                                Output:["_col1"]
+                                Group By Operator [GBY_184] (rows=381 width=178)
+                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                <-Union 19 [SIMPLE_EDGE]
+                                  <-Map 22 [CONTAINS] vectorized, llap
+                                    Reduce Output Operator [RS_210]
+                                      PartitionCols:_col0, _col1
+                                      Group By Operator [GBY_209] (rows=381 width=178)
+                                        Output:["_col0","_col1"],keys:_col1, _col0
+                                        Select Operator [SEL_208] (rows=500 width=178)
+                                          Output:["_col0","_col1"]
+                                          Filter Operator [FIL_207] (rows=500 width=178)
+                                            predicate:value is not null
+                                            TableScan [TS_45] (rows=500 width=178)
+                                              Output:["key","value"]
+                                  <-Reducer 18 [CONTAINS] vectorized, llap
+                                    Reduce Output Operator [RS_202]
+                                      PartitionCols:_col0, _col1
+                                      Group By Operator [GBY_201] (rows=381 width=178)
+                                        Output:["_col0","_col1"],keys:_col1, _col0
+                                        Select Operator [SEL_200] (rows=262 width=178)
+                                          Output:["_col0","_col1"]
+                                          Group By Operator [GBY_199] (rows=262 width=178)
+                                            Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                          <-Union 17 [SIMPLE_EDGE]
+                                            <-Map 16 [CONTAINS] vectorized, llap
+                                              Reduce Output Operator [RS_198]
+                                                PartitionCols:_col0, _col1
+                                                Group By Operator [GBY_197] (rows=262 width=178)
+                                                  Output:["_col0","_col1"],keys:_col1, _col0
+                                                  Select Operator [SEL_196] (rows=25 width=175)
+                                                    Output:["_col0","_col1"]
+                                                    Filter Operator [FIL_195] (rows=25 width=175)
+                                                      predicate:value is not null
+                                                      TableScan [TS_32] (rows=25 width=175)
+                                                        Output:["key","value"]
+                                            <-Map 21 [CONTAINS] vectorized, llap
+                                              Reduce Output Operator [RS_206]
+                                                PartitionCols:_col0, _col1
+                                                Group By Operator [GBY_205] (rows=262 width=178)
+                                                  Output:["_col0","_col1"],keys:_col1, _col0
+                                                  Select Operator [SEL_204] (rows=500 width=178)
+                                                    Output:["_col0","_col1"]
+                                                    Filter Operator [FIL_203] (rows=500 width=178)
+                                                      predicate:value is not null
+                                                      TableScan [TS_35] (rows=500 width=178)
+                                                        Output:["key","value"]
+                          <-Reducer 9 [SIMPLE_EDGE] llap
                             SHUFFLE [RS_58]
-                              PartitionCols:_col2
-                              Merge Join Operator [MERGEJOIN_167] (rows=30 width=86)
-                                Conds:RS_198._col1=RS_181._col1(Inner),Output:["_col2"]
+                              PartitionCols:_col3
+                              Merge Join Operator [MERGEJOIN_160] (rows=39 width=266)
+                                Conds:RS_168._col0=RS_172._col0(Inner),Output:["_col1","_col2","_col3"]
+                              <-Map 1 [SIMPLE_EDGE] vectorized, llap
+                                SHUFFLE [RS_168]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_166] (rows=500 width=178)
+                                    Output:["_col0","_col1"]
+                                    Filter Operator [FIL_165] (rows=500 width=178)
+                                      predicate:key is not null
+                                      TableScan [TS_0] (rows=500 width=178)
+                                        default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
                               <-Map 11 [SIMPLE_EDGE] vectorized, llap
-                                SHUFFLE [RS_181]
-                                  PartitionCols:_col1
-                                   Please refer to the previous Select Operator [SEL_179]
-                              <-Reducer 21 [SIMPLE_EDGE] vectorized, llap
-                                SHUFFLE [RS_198]
-                                  PartitionCols:_col1
-                                  Select Operator [SEL_197] (rows=381 width=178)
-                                    Output:["_col1"]
-                                    Group By Operator [GBY_196] (rows=381 width=178)
-                                      Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                    <-Union 20 [SIMPLE_EDGE]
-                                      <-Map 23 [CONTAINS] vectorized, llap
-                                        Reduce Output Operator [RS_220]
-                                          PartitionCols:_col0, _col1
-                                          Group By Operator [GBY_219] (rows=381 width=178)
-                                            Output:["_col0","_col1"],keys:_col1, _col0
-                                            Select Operator [SEL_218] (rows=500 width=178)
-                                              Output:["_col0","_col1"]
-                                              Filter Operator [FIL_217] (rows=500 width=178)
-                                                predicate:value is not null
-                                                TableScan [TS_39] (rows=500 width=178)
-                                                  Output:["key","value"]
-                                      <-Reducer 19 [CONTAINS] vectorized, llap
-                                        Reduce Output Operator [RS_212]
-                                          PartitionCols:_col0, _col1
-                                          Group By Operator [GBY_211] (rows=381 width=178)
-                                            Output:["_col0","_col1"],keys:_col1, _col0
-                                            Select Operator [SEL_210] (rows=262 width=178)
-                                              Output:["_col0","_col1"]
-                                              Group By Operator [GBY_209] (rows=262 width=178)
-                                                Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                              <-Union 18 [SIMPLE_EDGE]
-                                                <-Map 17 [CONTAINS] vectorized, llap
-                                                  Reduce Output Operator [RS_208]
-                                                    PartitionCols:_col0, _col1
-                                                    Group By Operator [GBY_207] (rows=262 width=178)
-                                                      Output:["_col0","_col1"],keys:_col1, _col0
-                                                      Select Operator [SEL_206] (rows=25 width=175)
-                                                        Output:["_col0","_col1"]
-                                                        Filter Operator [FIL_205] (rows=25 width=175)
-                                                          predicate:value is not null
-                                                          TableScan [TS_26] (rows=25 width=175)
-                                                            Output:["key","value"]
-                                                <-Map 22 [CONTAINS] vectorized, llap
-                                                  Reduce Output Operator [RS_216]
-                                                    PartitionCols:_col0, _col1
-                                                    Group By Operator [GBY_215] (rows=262 width=178)
-                                                      Output:["_col0","_col1"],keys:_col1, _col0
-                                                      Select Operator [SEL_214] (rows=500 width=178)
-                                                        Output:["_col0","_col1"]
-                                                        Filter Operator [FIL_213] (rows=500 width=178)
-                                                          predicate:value is not null
-                                                          TableScan [TS_29] (rows=500 width=178)
-                                                            Output:["key","value"]
-                  <-Reducer 5 [CONTAINS] llap
+                                SHUFFLE [RS_172]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_170] (rows=25 width=175)
+                                    Output:["_col0","_col1"]
+                                    Filter Operator [FIL_169] (rows=25 width=175)
+                                      predicate:(key is not null and value is not null)
+                                      TableScan [TS_3] (rows=25 width=175)
+                                        default@src1,x,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                  <-Reducer 3 [CONTAINS] llap
                     Reduce Output Operator [RS_66]
                       PartitionCols:_col0, _col1
                       Group By Operator [GBY_65] (rows=40 width=177)
                         Output:["_col0","_col1"],keys:_col0, _col1
                         Select Operator [SEL_25] (rows=33 width=177)
                           Output:["_col0","_col1"]
-                          Merge Join Operator [MERGEJOIN_166] (rows=33 width=177)
-                            Conds:RS_22._col2=RS_185._col0(Inner),Output:["_col2","_col5"]
-                          <-Map 16 [SIMPLE_EDGE] vectorized, llap
-                            SHUFFLE [RS_185]
-                              PartitionCols:_col0
-                               Please refer to the previous Select Operator [SEL_184]
-                          <-Reducer 4 [SIMPLE_EDGE] llap
+                          Merge Join Operator [MERGEJOIN_162] (rows=33 width=177)
+                            Conds:RS_22._col3=RS_175._col1(Inner),Output:["_col1","_col2"]
+                          <-Reducer 2 [SIMPLE_EDGE] llap
                             SHUFFLE [RS_22]
-                              PartitionCols:_col2
-                              Merge Join Operator [MERGEJOIN_165] (rows=21 width=86)
-                                Conds:RS_177._col1=RS_180._col1(Inner),Output:["_col2"]
+                              PartitionCols:_col3
+                              Merge Join Operator [MERGEJOIN_159] (rows=39 width=266)
+                                Conds:RS_167._col0=RS_171._col0(Inner),Output:["_col1","_col2","_col3"]
+                              <-Map 1 [SIMPLE_EDGE] vectorized, llap
+                                SHUFFLE [RS_167]
+                                  PartitionCols:_col0
+                                   Please refer to the previous Select Operator [SEL_166]
                               <-Map 11 [SIMPLE_EDGE] vectorized, llap
-                                SHUFFLE [RS_180]
-                                  PartitionCols:_col1
-                                   Please refer to the previous Select Operator [SEL_179]
-                              <-Reducer 3 [SIMPLE_EDGE] vectorized, llap
-                                SHUFFLE [RS_177]
-                                  PartitionCols:_col1
-                                  Select Operator [SEL_176] (rows=262 width=178)
-                                    Output:["_col1"]
-                                    Group By Operator [GBY_175] (rows=262 width=178)
-                                      Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                    <-Union 2 [SIMPLE_EDGE]
-                                      <-Map 1 [CONTAINS] vectorized, llap
-                                        Reduce Output Operator [RS_174]
-                                          PartitionCols:_col0, _col1
-                                          Group By Operator [GBY_173] (rows=262 width=178)
-                                            Output:["_col0","_col1"],keys:_col1, _col0
-                                            Select Operator [SEL_172] (rows=25 width=175)
-                                              Output:["_col0","_col1"]
-                                              Filter Operator [FIL_171] (rows=25 width=175)
-                                                predicate:value is not null
-                                                TableScan [TS_0] (rows=25 width=175)
-                                                  Output:["key","value"]
-                                      <-Map 10 [CONTAINS] vectorized, llap
-                                        Reduce Output Operator [RS_195]
-                                          PartitionCols:_col0, _col1
-                                          Group By Operator [GBY_194] (rows=262 width=178)
-                                            Output:["_col0","_col1"],keys:_col1, _col0
-                                            Select Operator [SEL_193] (rows=500 width=178)
-                                              Output:["_col0","_col1"]
-                                              Filter Operator [FIL_192] (rows=500 width=178)
-                                                predicate:value is not null
-                                                TableScan [TS_3] (rows=500 width=178)
-                                                  Output:["key","value"]
+                                SHUFFLE [RS_171]
+                                  PartitionCols:_col0
+                                   Please refer to the previous Select Operator [SEL_170]
+                          <-Reducer 14 [SIMPLE_EDGE] vectorized, llap
+                            SHUFFLE [RS_175]
+                              PartitionCols:_col1
+                              Select Operator [SEL_174] (rows=262 width=178)
+                                Output:["_col1"]
+                                Group By Operator [GBY_173] (rows=262 width=178)
+                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                <-Union 13 [SIMPLE_EDGE]
+                                  <-Map 12 [CONTAINS] vectorized, llap
+                                    Reduce Output Operator [RS_190]
+                                      PartitionCols:_col0, _col1
+                                      Group By Operator [GBY_189] (rows=262 width=178)
+                                        Output:["_col0","_col1"],keys:_col1, _col0
+                                        Select Operator [SEL_188] (rows=25 width=175)
+                                          Output:["_col0","_col1"]
+                                          Filter Operator [FIL_187] (rows=25 width=175)
+                                            predicate:value is not null
+                                            TableScan [TS_6] (rows=25 width=175)
+                                              Output:["key","value"]
+                                  <-Map 15 [CONTAINS] vectorized, llap
+                                    Reduce Output Operator [RS_194]
+                                      PartitionCols:_col0, _col1
+                                      Group By Operator [GBY_193] (rows=262 width=178)
+                                        Output:["_col0","_col1"],keys:_col1, _col0
+                                        Select Operator [SEL_192] (rows=500 width=178)
+                                          Output:["_col0","_col1"]
+                                          Filter Operator [FIL_191] (rows=500 width=178)
+                                            predicate:value is not null
+                                            TableScan [TS_9] (rows=500 width=178)
+                                              Output:["key","value"]
+          <-Reducer 8 [CONTAINS] llap
+            Reduce Output Operator [RS_119]
+              PartitionCols:_col0, _col1
+              Group By Operator [GBY_118] (rows=47 width=177)
+                Output:["_col0","_col1"],keys:_col0, _col1
+                Select Operator [SEL_114] (rows=55 width=177)
+                  Output:["_col0","_col1"]
+                  Merge Join Operator [MERGEJOIN_164] (rows=55 width=177)
+                    Conds:RS_111._col3=RS_183._col1(Inner),Output:["_col1","_col2"]
+                  <-Reducer 2 [SIMPLE_EDGE] llap
+                    SHUFFLE [RS_111]
+                      PartitionCols:_col3
+                       Please refer to the previous Merge Join Operator [MERGEJOIN_159]
+                  <-Reducer 29 [SIMPLE_EDGE] vectorized, llap
+                    SHUFFLE [RS_183]
+                      PartitionCols:_col1
+                      Select Operator [SEL_182] (rows=440 width=178)
+                        Output:["_col1"]
+                        Group By Operator [GBY_181] (rows=440 width=178)
+                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                        <-Union 28 [SIMPLE_EDGE]
+                          <-Map 32 [CONTAINS] vectorized, llap
+                            Reduce Output Operator [RS_234]
+                              PartitionCols:_col0, _col1
+                              Group By Operator [GBY_233] (rows=440 width=178)
+                                Output:["_col0","_col1"],keys:_col1, _col0
+                                Select Operator [SEL_232] (rows=500 width=178)
+                                  Output:["_col0","_col1"]
+                                  Filter Operator [FIL_231] (rows=500 width=178)
+                                    predicate:value is not null
+                                    TableScan [TS_98] (rows=500 width=178)
+                                      Output:["key","value"]
+                          <-Reducer 27 [CONTAINS] vectorized, llap
+                            Reduce Output Operator [RS_222]
+                              PartitionCols:_col0, _col1
+                              Group By Operator [GBY_221] (rows=440 width=178)
+                                Output:["_col0","_col1"],keys:_col1, _col0
+                                Select Operator [SEL_220] (rows=381 width=178)
+                                  Output:["_col0","_col1"]
+                                  Group By Operator [GBY_219] (rows=381 width=178)
+                                    Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                  <-Union 26 [SIMPLE_EDGE]
+                                    <-Map 31 [CONTAINS] vectorized, llap
+                                      Reduce Output Operator [RS_230]
+                                        PartitionCols:_col0, _col1
+                                        Group By Operator [GBY_229] (rows=381 width=178)
+                                          Output:["_col0","_col1"],keys:_col1, _col0
+                                          Select Operator [SEL_228] (rows=500 width=178)
+                                            Output:["_col0","_col1"]
+                                            Filter Operator [FIL_227] (rows=500 width=178)
+                                              predicate:value is not null
+                                              TableScan [TS_88] (rows=500 width=178)
+                                                Output:["key","value"]
+                                    <-Reducer 25 [CONTAINS] vectorized, llap
+                                      Reduce Output Operator [RS_218]
+                                        PartitionCols:_col0, _col1
+                                        Group By Operator [GBY_217] (rows=381 width=178)
+                                          Output:["_col0","_col1"],keys:_col1, _col0
+                                          Select Operator [SEL_216] (rows=262 width=178)
+                                            Output:["_col0","_col1"]
+                                            Group By Operator [GBY_215] (rows=262 width=178)
+                                              Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                            <-Union 24 [SIMPLE_EDGE]
+                                              <-Map 23 [CONTAINS] vectorized, llap
+                                                Reduce Output Operator [RS_214]
+                                                  PartitionCols:_col0, _col1
+                                                  Group By Operator [GBY_213] (rows=262 width=178)
+                                                    Output:["_col0","_col1"],keys:_col1, _col0
+                                                    Select Operator [SEL_212] (rows=25 width=175)
+                                                      Output:["_col0","_col1"]
+                                                      Filter Operator [FIL_211] (rows=25 width=175)
+                                                        predicate:value is not null
+                                                        TableScan [TS_75] (rows=25 width=175)
+                                                          Output:["key","value"]
+                                              <-Map 30 [CONTAINS] vectorized, llap
+                                                Reduce Output Operator [RS_226]
+                                                  PartitionCols:_col0, _col1
+                                                  Group By Operator [GBY_225] (rows=262 width=178)
+                                                    Output:["_col0","_col1"],keys:_col1, _col0
+                                                    Select Operator [SEL_224] (rows=500 width=178)
+                                                      Output:["_col0","_col1"]
+                                                      Filter Operator [FIL_223] (rows=500 width=178)
+                                                        predicate:value is not null
+                                                        TableScan [TS_78] (rows=500 width=178)
+                                                          Output:["key","value"]
 
 PREHOOK: query: EXPLAIN
 SELECT x.key, z.value, y.value
@@ -1133,135 +1113,132 @@ POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Union 2 (CONTAINS)
-Map 12 <- Union 10 (CONTAINS)
-Map 6 <- Union 2 (CONTAINS)
-Map 9 <- Union 10 (CONTAINS)
-Reducer 11 <- Map 13 (BROADCAST_EDGE), Map 14 (BROADCAST_EDGE), Union 10 (SIMPLE_EDGE), Union 4 (CONTAINS)
-Reducer 3 <- Map 7 (BROADCAST_EDGE), Map 8 (BROADCAST_EDGE), Union 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
-Reducer 5 <- Union 4 (SIMPLE_EDGE)
+Map 1 <- Map 2 (BROADCAST_EDGE)
+Map 10 <- Union 11 (CONTAINS)
+Map 13 <- Union 11 (CONTAINS)
+Map 3 <- Union 4 (CONTAINS)
+Map 8 <- Union 4 (CONTAINS)
+Map 9 <- Map 2 (BROADCAST_EDGE)
+Reducer 12 <- Map 9 (BROADCAST_EDGE), Union 11 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 5 <- Map 1 (BROADCAST_EDGE), Union 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 7 <- Union 6 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 5 vectorized, llap
-      File Output Operator [FS_108]
-        Group By Operator [GBY_107] (rows=550 width=10)
+      Reducer 7 vectorized, llap
+      File Output Operator [FS_107]
+        Group By Operator [GBY_106] (rows=605 width=10)
           Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-        <-Union 4 [SIMPLE_EDGE]
-          <-Reducer 11 [CONTAINS] vectorized, llap
-            Reduce Output Operator [RS_129]
+        <-Union 6 [SIMPLE_EDGE]
+          <-Reducer 12 [CONTAINS] vectorized, llap
+            Reduce Output Operator [RS_125]
               PartitionCols:_col0, _col1
-              Group By Operator [GBY_128] (rows=1100 width=10)
+              Group By Operator [GBY_124] (rows=1210 width=10)
                 Output:["_col0","_col1"],keys:_col0, _col1
-                Select Operator [SEL_127] (rows=550 width=10)
+                Select Operator [SEL_123] (rows=605 width=10)
                   Output:["_col0","_col1"]
-                  Map Join Operator [MAPJOIN_126] (rows=550 width=10)
-                    Conds:MAPJOIN_125._col2=RS_122._col0(Inner),Output:["_col1","_col2"]
-                  <-Map 14 [BROADCAST_EDGE] vectorized, llap
-                    BROADCAST [RS_122]
-                      PartitionCols:_col0
-                      Select Operator [SEL_121] (rows=500 width=10)
-                        Output:["_col0"]
-                        Filter Operator [FIL_120] (rows=500 width=10)
-                          predicate:key is not null
-                          TableScan [TS_42] (rows=500 width=10)
-                            default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key"]
-                  <-Map Join Operator [MAPJOIN_125] (rows=288 width=10)
-                      Conds:SEL_124._col1=RS_119._col1(Inner),Output:["_col1","_col2"]
-                    <-Map 13 [BROADCAST_EDGE] vectorized, llap
-                      BROADCAST [RS_119]
-                        PartitionCols:_col1
-                        Select Operator [SEL_118] (rows=25 width=7)
-                          Output:["_col0","_col1"]
-                          Filter Operator [FIL_117] (rows=25 width=7)
-                            predicate:(key is not null and value is not null)
-                            TableScan [TS_39] (rows=25 width=7)
-                              default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                    <-Select Operator [SEL_124] (rows=262 width=10)
-                        Output:["_col1"]
-                        Group By Operator [GBY_123] (rows=262 width=10)
-                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                        <-Union 10 [SIMPLE_EDGE]
-                          <-Map 12 [CONTAINS] vectorized, llap
-                            Reduce Output Operator [RS_133]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_132] (rows=525 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_131] (rows=500 width=10)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_130] (rows=500 width=10)
-                                    predicate:value is not null
-                                    TableScan [TS_29] (rows=500 width=10)
-                                      Output:["key","value"]
-                          <-Map 9 [CONTAINS] vectorized, llap
-                            Reduce Output Operator [RS_116]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_115] (rows=525 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_114] (rows=25 width=7)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_113] (rows=25 width=7)
-                                    predicate:value is not null
-                                    TableScan [TS_26] (rows=25 width=7)
-                                      Output:["key","value"]
-          <-Reducer 3 [CONTAINS] vectorized, llap
-            Reduce Output Operator [RS_106]
+                  Map Join Operator [MAPJOIN_122] (rows=605 width=10)
+                    Conds:RS_119._col2=SEL_121._col1(Inner),Output:["_col1","_col4"]
+                  <-Map 9 [BROADCAST_EDGE] vectorized, llap
+                    BROADCAST [RS_119]
+                      PartitionCols:_col2
+                      Map Join Operator [MAPJOIN_118] (rows=550 width=10)
+                        Conds:SEL_117._col0=RS_95._col0(Inner),Output:["_col1","_col2"]
+                      <-Map 2 [BROADCAST_EDGE] vectorized, llap
+                        BROADCAST [RS_95]
+                          PartitionCols:_col0
+                          Select Operator [SEL_93] (rows=25 width=7)
+                            Output:["_col0","_col1"]
+                            Filter Operator [FIL_92] (rows=25 width=7)
+                              predicate:(key is not null and value is not null)
+                              TableScan [TS_3] (rows=25 width=7)
+                                default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
+                      <-Select Operator [SEL_117] (rows=500 width=10)
+                          Output:["_col0"]
+                          Filter Operator [FIL_116] (rows=500 width=10)
+                            predicate:key is not null
+                            TableScan [TS_26] (rows=500 width=10)
+                              default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key"]
+                  <-Select Operator [SEL_121] (rows=262 width=10)
+                      Output:["_col1"]
+                      Group By Operator [GBY_120] (rows=262 width=10)
+                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                      <-Union 11 [SIMPLE_EDGE]
+                        <-Map 10 [CONTAINS] vectorized, llap
+                          Reduce Output Operator [RS_115]
+                            PartitionCols:_col0, _col1
+                            Group By Operator [GBY_114] (rows=525 width=10)
+                              Output:["_col0","_col1"],keys:_col1, _col0
+                              Select Operator [SEL_113] (rows=25 width=7)
+                                Output:["_col0","_col1"]
+                                Filter Operator [FIL_112] (rows=25 width=7)
+                                  predicate:value is not null
+                                  TableScan [TS_32] (rows=25 width=7)
+                                    Output:["key","value"]
+                        <-Map 13 [CONTAINS] vectorized, llap
+                          Reduce Output Operator [RS_129]
+                            PartitionCols:_col0, _col1
+                            Group By Operator [GBY_128] (rows=525 width=10)
+                              Output:["_col0","_col1"],keys:_col1, _col0
+                              Select Operator [SEL_127] (rows=500 width=10)
+                                Output:["_col0","_col1"]
+                                Filter Operator [FIL_126] (rows=500 width=10)
+                                  predicate:value is not null
+                                  TableScan [TS_35] (rows=500 width=10)
+                                    Output:["key","value"]
+          <-Reducer 5 [CONTAINS] vectorized, llap
+            Reduce Output Operator [RS_105]
               PartitionCols:_col0, _col1
-              Group By Operator [GBY_105] (rows=1100 width=10)
+              Group By Operator [GBY_104] (rows=1210 width=10)
                 Output:["_col0","_col1"],keys:_col0, _col1
-                Select Operator [SEL_104] (rows=550 width=10)
+                Select Operator [SEL_103] (rows=605 width=10)
                   Output:["_col0","_col1"]
-                  Map Join Operator [MAPJOIN_103] (rows=550 width=10)
-                    Conds:MAPJOIN_102._col2=RS_99._col0(Inner),Output:["_col1","_col2"]
-                  <-Map 8 [BROADCAST_EDGE] vectorized, llap
+                  Map Join Operator [MAPJOIN_102] (rows=605 width=10)
+                    Conds:RS_99._col2=SEL_101._col1(Inner),Output:["_col1","_col4"]
+                  <-Map 1 [BROADCAST_EDGE] vectorized, llap
                     BROADCAST [RS_99]
-                      PartitionCols:_col0
-                      Select Operator [SEL_98] (rows=500 width=10)
-                        Output:["_col0"]
-                        Filter Operator [FIL_97] (rows=500 width=10)
-                          predicate:key is not null
-                          TableScan [TS_16] (rows=500 width=10)
-                            default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key"]
-                  <-Map Join Operator [MAPJOIN_102] (rows=288 width=10)
-                      Conds:SEL_101._col1=RS_96._col1(Inner),Output:["_col1","_col2"]
-                    <-Map 7 [BROADCAST_EDGE] vectorized, llap
-                      BROADCAST [RS_96]
-                        PartitionCols:_col1
-                        Select Operator [SEL_95] (rows=25 width=7)
-                          Output:["_col0","_col1"]
-                          Filter Operator [FIL_94] (rows=25 width=7)
-                            predicate:(key is not null and value is not null)
-                            TableScan [TS_13] (rows=25 width=7)
-                              default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                    <-Select Operator [SEL_101] (rows=262 width=10)
-                        Output:["_col1"]
-                        Group By Operator [GBY_100] (rows=262 width=10)
-                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                        <-Union 2 [SIMPLE_EDGE]
-                          <-Map 1 [CONTAINS] vectorized, llap
-                            Reduce Output Operator [RS_93]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_92] (rows=525 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_91] (rows=25 width=7)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_90] (rows=25 width=7)
-                                    predicate:value is not null
-                                    TableScan [TS_0] (rows=25 width=7)
-                                      Output:["key","value"]
-                          <-Map 6 [CONTAINS] vectorized, llap
-                            Reduce Output Operator [RS_112]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_111] (rows=525 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_110] (rows=500 width=10)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_109] (rows=500 width=10)
-                                    predicate:value is not null
-                                    TableScan [TS_3] (rows=500 width=10)
-                                      Output:["key","value"]
+                      PartitionCols:_col2
+                      Map Join Operator [MAPJOIN_98] (rows=550 width=10)
+                        Conds:SEL_97._col0=RS_94._col0(Inner),Output:["_col1","_col2"]
+                      <-Map 2 [BROADCAST_EDGE] vectorized, llap
+                        BROADCAST [RS_94]
+                          PartitionCols:_col0
+                           Please refer to the previous Select Operator [SEL_93]
+                      <-Select Operator [SEL_97] (rows=500 width=10)
+                          Output:["_col0"]
+                          Filter Operator [FIL_96] (rows=500 width=10)
+                            predicate:key is not null
+                            TableScan [TS_0] (rows=500 width=10)
+                              default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key"]
+                  <-Select Operator [SEL_101] (rows=262 width=10)
+                      Output:["_col1"]
+                      Group By Operator [GBY_100] (rows=262 width=10)
+                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                      <-Union 4 [SIMPLE_EDGE]
+                        <-Map 3 [CONTAINS] vectorized, llap
+                          Reduce Output Operator [RS_91]
+                            PartitionCols:_col0, _col1
+                            Group By Operator [GBY_90] (rows=525 width=10)
+                              Output:["_col0","_col1"],keys:_col1, _col0
+                              Select Operator [SEL_89] (rows=25 width=7)
+                                Output:["_col0","_col1"]
+                                Filter Operator [FIL_88] (rows=25 width=7)
+                                  predicate:value is not null
+                                  TableScan [TS_6] (rows=25 width=7)
+                                    Output:["key","value"]
+                        <-Map 8 [CONTAINS] vectorized, llap
+                          Reduce Output Operator [RS_111]
+                            PartitionCols:_col0, _col1
+                            Group By Operator [GBY_110] (rows=525 width=10)
+                              Output:["_col0","_col1"],keys:_col1, _col0
+                              Select Operator [SEL_109] (rows=500 width=10)
+                                Output:["_col0","_col1"]
+                                Filter Operator [FIL_108] (rows=500 width=10)
+                                  predicate:value is not null
+                                  TableScan [TS_9] (rows=500 width=10)
+                                    Output:["key","value"]
 
 PREHOOK: query: explain
 SELECT x.key, y.value
@@ -1292,262 +1269,253 @@ POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Union 2 (CONTAINS)
-Map 11 <- Union 12 (CONTAINS)
-Map 16 <- Union 12 (CONTAINS)
-Map 17 <- Union 14 (CONTAINS)
-Map 20 <- Union 21 (CONTAINS)
-Map 27 <- Union 21 (CONTAINS)
-Map 28 <- Union 23 (CONTAINS)
-Map 29 <- Union 25 (CONTAINS)
-Map 8 <- Union 2 (CONTAINS)
-Reducer 13 <- Union 12 (SIMPLE_EDGE), Union 14 (CONTAINS)
-Reducer 15 <- Map 18 (BROADCAST_EDGE), Map 19 (BROADCAST_EDGE), Union 14 (SIMPLE_EDGE), Union 4 (CONTAINS)
-Reducer 22 <- Union 21 (SIMPLE_EDGE), Union 23 (CONTAINS)
-Reducer 24 <- Union 23 (SIMPLE_EDGE), Union 25 (CONTAINS)
-Reducer 26 <- Map 10 (BROADCAST_EDGE), Map 9 (BROADCAST_EDGE), Union 25 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 3 <- Map 10 (BROADCAST_EDGE), Map 9 (BROADCAST_EDGE), Union 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
-Reducer 5 <- Union 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 7 <- Union 6 (SIMPLE_EDGE)
+Map 1 <- Map 2 (BROADCAST_EDGE)
+Map 10 <- Union 4 (CONTAINS)
+Map 11 <- Map 2 (BROADCAST_EDGE)
+Map 12 <- Union 13 (CONTAINS)
+Map 17 <- Union 13 (CONTAINS)
+Map 18 <- Union 15 (CONTAINS)
+Map 19 <- Union 20 (CONTAINS)
+Map 26 <- Union 20 (CONTAINS)
+Map 27 <- Union 22 (CONTAINS)
+Map 28 <- Union 24 (CONTAINS)
+Map 3 <- Union 4 (CONTAINS)
+Reducer 14 <- Union 13 (SIMPLE_EDGE), Union 15 (CONTAINS)
+Reducer 16 <- Map 11 (BROADCAST_EDGE), Union 15 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 21 <- Union 20 (SIMPLE_EDGE), Union 22 (CONTAINS)
+Reducer 23 <- Union 22 (SIMPLE_EDGE), Union 24 (CONTAINS)
+Reducer 25 <- Map 1 (BROADCAST_EDGE), Union 24 (SIMPLE_EDGE), Union 8 (CONTAINS)
+Reducer 5 <- Map 1 (BROADCAST_EDGE), Union 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 7 <- Union 6 (SIMPLE_EDGE), Union 8 (CONTAINS)
+Reducer 9 <- Union 8 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 7 vectorized, llap
-      File Output Operator [FS_200]
-        Group By Operator [GBY_199] (rows=550 width=10)
+      Reducer 9 vectorized, llap
+      File Output Operator [FS_193]
+        Group By Operator [GBY_192] (rows=605 width=10)
           Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-        <-Union 6 [SIMPLE_EDGE]
-          <-Reducer 26 [CONTAINS] vectorized, llap
-            Reduce Output Operator [RS_252]
+        <-Union 8 [SIMPLE_EDGE]
+          <-Reducer 25 [CONTAINS] vectorized, llap
+            Reduce Output Operator [RS_241]
               PartitionCols:_col0, _col1
-              Group By Operator [GBY_251] (rows=1100 width=10)
+              Group By Operator [GBY_240] (rows=1210 width=10)
                 Output:["_col0","_col1"],keys:_col0, _col1
-                Select Operator [SEL_250] (rows=550 width=10)
+                Select Operator [SEL_239] (rows=605 width=10)
                   Output:["_col0","_col1"]
-                  Map Join Operator [MAPJOIN_249] (rows=550 width=10)
-                    Conds:MAPJOIN_248._col2=RS_188._col0(Inner),Output:["_col2","_col5"]
-                  <-Map 10 [BROADCAST_EDGE] vectorized, llap
-                    BROADCAST [RS_188]
-                      PartitionCols:_col0
-                      Select Operator [SEL_186] (rows=500 width=10)
-                        Output:["_col0","_col1"]
-                        Filter Operator [FIL_185] (rows=500 width=10)
-                          predicate:key is not null
-                          TableScan [TS_16] (rows=500 width=10)
-                            default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                  <-Map Join Operator [MAPJOIN_248] (rows=484 width=10)
-                      Conds:SEL_247._col1=RS_184._col1(Inner),Output:["_col2"]
-                    <-Map 9 [BROADCAST_EDGE] vectorized, llap
-                      BROADCAST [RS_184]
-                        PartitionCols:_col1
-                        Select Operator [SEL_182] (rows=25 width=7)
-                          Output:["_col0","_col1"]
-                          Filter Operator [FIL_181] (rows=25 width=7)
-                            predicate:(key is not null and value is not null)
-                            TableScan [TS_13] (rows=25 width=7)
-                              default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                    <-Select Operator [SEL_247] (rows=440 width=10)
-                        Output:["_col1"]
-                        Group By Operator [GBY_246] (rows=440 width=10)
-                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                        <-Union 25 [SIMPLE_EDGE]
-                          <-Map 29 [CONTAINS] vectorized, llap
-                            Reduce Output Operator [RS_264]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_263] (rows=881 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_262] (rows=500 width=10)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_261] (rows=500 width=10)
-                                    predicate:value is not null
-                                    TableScan [TS_92] (rows=500 width=10)
-                                      Output:["key","value"]
-                          <-Reducer 24 [CONTAINS] vectorized, llap
-                            Reduce Output Operator [RS_245]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_244] (rows=881 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_243] (rows=381 width=10)
-                                  Output:["_col0","_col1"]
-                                  Group By Operator [GBY_242] (rows=381 width=10)
-                                    Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                  <-Union 23 [SIMPLE_EDGE]
-                                    <-Map 28 [CONTAINS] vectorized, llap
-                                      Reduce Output Operator [RS_260]
-                                        PartitionCols:_col0, _col1
-                                        Group By Operator [GBY_259] (rows=762 width=10)
-                                          Output:["_col0","_col1"],keys:_col1, _col0
-                                          Select Operator [SEL_258] (rows=500 width=10)
-                                            Output:["_col0","_col1"]
-                                            Filter Operator [FIL_257] (rows=500 width=10)
-                                              predicate:value is not null
-                                              TableScan [TS_82] (rows=500 width=10)
-                                                Output:["key","value"]
-                                    <-Reducer 22 [CONTAINS] vectorized, llap
-                                      Reduce Output Operator [RS_241]
-                                        PartitionCols:_col0, _col1
-                                        Group By Operator [GBY_240] (rows=762 width=10)
-                                          Output:["_col0","_col1"],keys:_col1, _col0
-                                          Select Operator [SEL_239] (rows=262 width=10)
-                                            Output:["_col0","_col1"]
-                                            Group By Operator [GBY_238] (rows=262 width=10)
-                                              Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                            <-Union 21 [SIMPLE_EDGE]
-                                              <-Map 20 [CONTAINS] vectorized, llap
-                                                Reduce Output Operator [RS_237]
-                                                  PartitionCols:_col0, _col1
-                                                  Group By Operator [GBY_236] (rows=525 width=10)
-                                                    Output:["_col0","_col1"],keys:_col1, _col0
-                                                    Select Operator [SEL_235] (rows=25 width=7)
-                                                      Output:["_col0","_col1"]
-                                                      Filter Operator [FIL_234] (rows=25 width=7)
-                                                        predicate:value is not null
-                                                        TableScan [TS_69] (rows=25 width=7)
-                                                          Output:["key","value"]
-                                              <-Map 27 [CONTAINS] vectorized, llap
-                                                Reduce Output Operator [RS_256]
-                                                  PartitionCols:_col0, _col1
-                                                  Group By Operator [GBY_255] (rows=525 width=10)
-                                                    Output:["_col0","_col1"],keys:_col1, _col0
-                                                    Select Operator [SEL_254] (rows=500 width=10)
-                                                      Output:["_col0","_col1"]
-                                                      Filter Operator [FIL_253] (rows=500 width=10)
-                                                        predicate:value is not null
-                                                        TableScan [TS_72] (rows=500 width=10)
-                                                          Output:["key","value"]
-          <-Reducer 5 [CONTAINS] vectorized, llap
-            Reduce Output Operator [RS_198]
-              PartitionCols:_col0, _col1
-              Group By Operator [GBY_197] (rows=1100 width=10)
-                Output:["_col0","_col1"],keys:_col0, _col1
-                Group By Operator [GBY_196] (rows=550 width=10)
-                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                <-Union 4 [SIMPLE_EDGE]
-                  <-Reducer 15 [CONTAINS] vectorized, llap
-                    Reduce Output Operator [RS_225]
-                      PartitionCols:_col0, _col1
-                      Group By Operator [GBY_224] (rows=1100 width=10)
-                        Output:["_col0","_col1"],keys:_col0, _col1
-                        Select Operator [SEL_223] (rows=550 width=10)
+                  Map Join Operator [MAPJOIN_238] (rows=605 width=10)
+                    Conds:RS_182._col3=SEL_237._col1(Inner),Output:["_col1","_col2"]
+                  <-Map 1 [BROADCAST_EDGE] vectorized, llap
+                    BROADCAST [RS_182]
+                      PartitionCols:_col3
+                      Map Join Operator [MAPJOIN_180] (rows=550 width=10)
+                        Conds:SEL_179._col0=RS_176._col0(Inner),Output:["_col1","_col2","_col3"]
+                      <-Map 2 [BROADCAST_EDGE] vectorized, llap
+                        BROADCAST [RS_176]
+                          PartitionCols:_col0
+                          Select Operator [SEL_175] (rows=25 width=7)
+                            Output:["_col0","_col1"]
+                            Filter Operator [FIL_174] (rows=25 width=7)
+                              predicate:(key is not null and value is not null)
+                              TableScan [TS_3] (rows=25 width=7)
+                                default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
+                      <-Select Operator [SEL_179] (rows=500 width=10)
                           Output:["_col0","_col1"]
-                          Map Join Operator [MAPJOIN_222] (rows=550 width=10)
-                            Conds:MAPJOIN_221._col2=RS_218._col0(Inner),Output:["_col2","_col5"]
-                          <-Map 19 [BROADCAST_EDGE] vectorized, llap
-                            BROADCAST [RS_218]
-                              PartitionCols:_col0
-                              Select Operator [SEL_217] (rows=500 width=10)
+                          Filter Operator [FIL_178] (rows=500 width=10)
+                            predicate:key is not null
+                            TableScan [TS_0] (rows=500 width=10)
+                              default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
+                  <-Select Operator [SEL_237] (rows=440 width=10)
+                      Output:["_col1"]
+                      Group By Operator [GBY_236] (rows=440 width=10)
+                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                      <-Union 24 [SIMPLE_EDGE]
+                        <-Map 28 [CONTAINS] vectorized, llap
+                          Reduce Output Operator [RS_253]
+                            PartitionCols:_col0, _col1
+                            Group By Operator [GBY_252] (rows=881 width=10)
+                              Output:["_col0","_col1"],keys:_col1, _col0
+                              Select Operator [SEL_251] (rows=500 width=10)
                                 Output:["_col0","_col1"]
-                                Filter Operator [FIL_216] (rows=500 width=10)
-                                  predicate:key is not null
-                                  TableScan [TS_52] (rows=500 width=10)
-                                    default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                          <-Map Join Operator [MAPJOIN_221] (rows=419 width=10)
-                              Conds:SEL_220._col1=RS_215._col1(Inner),Output:["_col2"]
-                            <-Map 18 [BROADCAST_EDGE] vectorized, llap
-                              BROADCAST [RS_215]
-                                PartitionCols:_col1
-                                Select Operator [SEL_214] (rows=25 width=7)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_213] (rows=25 width=7)
-                                    predicate:(key is not null and value is not null)
-                                    TableScan [TS_49] (rows=25 width=7)
-                                      default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                            <-Select Operator [SEL_220] (rows=381 width=10)
-                                Output:["_col1"]
-                                Group By Operator [GBY_219] (rows=381 width=10)
+                                Filter Operator [FIL_250] (rows=500 width=10)
+                                  predicate:value is not null
+                                  TableScan [TS_98] (rows=500 width=10)
+                                    Output:["key","value"]
+                        <-Reducer 23 [CONTAINS] vectorized, llap
+                          Reduce Output Operator [RS_235]
+                            PartitionCols:_col0, _col1
+                            Group By Operator [GBY_234] (rows=881 width=10)
+                              Output:["_col0","_col1"],keys:_col1, _col0
+                              Select Operator [SEL_233] (rows=381 width=10)
+                                Output:["_col0","_col1"]
+                                Group By Operator [GBY_232] (rows=381 width=10)
                                   Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                <-Union 14 [SIMPLE_EDGE]
-                                  <-Map 17 [CONTAINS] vectorized, llap
-                                    Reduce Output Operator [RS_233]
+                                <-Union 22 [SIMPLE_EDGE]
+                                  <-Map 27 [CONTAINS] vectorized, llap
+                                    Reduce Output Operator [RS_249]
                                       PartitionCols:_col0, _col1
-                                      Group By Operator [GBY_232] (rows=762 width=10)
+                                      Group By Operator [GBY_248] (rows=762 width=10)
                                         Output:["_col0","_col1"],keys:_col1, _col0
-                                        Select Operator [SEL_231] (rows=500 width=10)
+                                        Select Operator [SEL_247] (rows=500 width=10)
                                           Output:["_col0","_col1"]
-                                          Filter Operator [FIL_230] (rows=500 width=10)
+                                          Filter Operator [FIL_246] (rows=500 width=10)
                                             predicate:value is not null
-                                            TableScan [TS_39] (rows=500 width=10)
+                                            TableScan [TS_88] (rows=500 width=10)
                                               Output:["key","value"]
-                                  <-Reducer 13 [CONTAINS] vectorized, llap
-                                    Reduce Output Operator [RS_212]
+                                  <-Reducer 21 [CONTAINS] vectorized, llap
+                                    Reduce Output Operator [RS_231]
                                       PartitionCols:_col0, _col1
-                                      Group By Operator [GBY_211] (rows=762 width=10)
+                                      Group By Operator [GBY_230] (rows=762 width=10)
                                         Output:["_col0","_col1"],keys:_col1, _col0
-                                        Select Operator [SEL_210] (rows=262 width=10)
+                                        Select Operator [SEL_229] (rows=262 width=10)
                                           Output:["_col0","_col1"]
-                                          Group By Operator [GBY_209] (rows=262 width=10)
+                                          Group By Operator [GBY_228] (rows=262 width=10)
                                             Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                          <-Union 12 [SIMPLE_EDGE]
-                                            <-Map 11 [CONTAINS] vectorized, llap
-                                              Reduce Output Operator [RS_208]
+                                          <-Union 20 [SIMPLE_EDGE]
+                                            <-Map 19 [CONTAINS] vectorized, llap
+                                              Reduce Output Operator [RS_227]
                                                 PartitionCols:_col0, _col1
-                                                Group By Operator [GBY_207] (rows=525 width=10)
+                                                Group By Operator [GBY_226] (rows=525 width=10)
                                                   Output:["_col0","_col1"],keys:_col1, _col0
-                                                  Select Operator [SEL_206] (rows=25 width=7)
+                                                  Select Operator [SEL_225] (rows=25 width=7)
                                                     Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_205] (rows=25 width=7)
+                                                    Filter Operator [FIL_224] (rows=25 width=7)
                                                       predicate:value is not null
-                                                      TableScan [TS_26] (rows=25 width=7)
+                                                      TableScan [TS_75] (rows=25 width=7)
                                                         Output:["key","value"]
-                                            <-Map 16 [CONTAINS] vectorized, llap
-                                              Reduce Output Operator [RS_229]
+                                            <-Map 26 [CONTAINS] vectorized, llap
+                                              Reduce Output Operator [RS_245]
                                                 PartitionCols:_col0, _col1
-                                                Group By Operator [GBY_228] (rows=525 width=10)
+                                                Group By Operator [GBY_244] (rows=525 width=10)
                                                   Output:["_col0","_col1"],keys:_col1, _col0
-                                                  Select Operator [SEL_227] (rows=500 width=10)
+                                                  Select Operator [SEL_243] (rows=500 width=10)
                                                     Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_226] (rows=500 width=10)
+                                                    Filter Operator [FIL_242] (rows=500 width=10)
                                                       predicate:value is not null
-                                                      T

<TRUNCATED>

[04/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out b/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
index b94b018..ff74455 100644
--- a/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
@@ -67,12 +67,12 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col1 (type: string)
-                  Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -84,11 +84,11 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 500 Data size: 100570 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 100392 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
-                  Statistics: Num rows: 500 Data size: 100570 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 100392 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -96,10 +96,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 500 Data size: 100570 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 100392 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 500 Data size: 100570 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 100392 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out b/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
index b5e07d7..012b934 100644
--- a/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
@@ -67,12 +67,12 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col1 (type: string)
-                  Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -84,11 +84,11 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 500 Data size: 100570 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 100392 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
-                  Statistics: Num rows: 500 Data size: 100570 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 100392 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -96,10 +96,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 500 Data size: 100570 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 100392 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 500 Data size: 100570 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 100392 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out b/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
index 2f98950..468f45c 100644
--- a/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
@@ -66,10 +66,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 40 Data size: 14120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 13767 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 40 Data size: 14120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 13767 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/tez_union.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_union.q.out b/ql/src/test/results/clientpositive/llap/tez_union.q.out
index 65e2de2..5e5e9a8 100644
--- a/ql/src/test/results/clientpositive/llap/tez_union.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_union.q.out
@@ -42,10 +42,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 1309 Data size: 233002 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1291 Data size: 229798 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -82,7 +82,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
-                      Statistics: Num rows: 1309 Data size: 233002 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1291 Data size: 229798 Basic stats: COMPLETE Column stats: COMPLETE
                       table:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -264,7 +264,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
-                Statistics: Num rows: 3236 Data size: 25888 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 3164 Data size: 25312 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -400,10 +400,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 1618 Data size: 281532 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1582 Data size: 275268 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 1618 Data size: 281532 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1582 Data size: 275268 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -431,10 +431,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 1618 Data size: 281532 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1582 Data size: 275268 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 1618 Data size: 281532 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1582 Data size: 275268 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -556,7 +556,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 4
-                        Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
@@ -569,15 +569,15 @@ STAGE PLANS:
                           input vertices:
                             1 Map 9
                             2 Map 10
-                          Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                           Select Operator
                             expressions: _col1 (type: string), _col0 (type: string), _col2 (type: string)
                             outputColumnNames: _col0, _col1, _col2
-                            Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                             Reduce Output Operator
                               key expressions: _col2 (type: string)
                               sort order: +
-                              Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                               value expressions: _col0 (type: string), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -650,7 +650,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 6
-                        Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
@@ -663,15 +663,15 @@ STAGE PLANS:
                           input vertices:
                             1 Map 9
                             2 Map 10
-                          Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                           Select Operator
                             expressions: _col1 (type: string), _col0 (type: string), _col2 (type: string)
                             outputColumnNames: _col0, _col1, _col2
-                            Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                             Reduce Output Operator
                               key expressions: _col2 (type: string)
                               sort order: +
-                              Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                               value expressions: _col0 (type: string), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -715,7 +715,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 8
-                        Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
@@ -728,15 +728,15 @@ STAGE PLANS:
                           input vertices:
                             1 Map 9
                             2 Map 10
-                          Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                           Select Operator
                             expressions: _col1 (type: string), _col0 (type: string), _col2 (type: string)
                             outputColumnNames: _col0, _col1, _col2
-                            Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                             Reduce Output Operator
                               key expressions: _col2 (type: string)
                               sort order: +
-                              Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                               value expressions: _col0 (type: string), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -794,10 +794,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 6354 Data size: 1658394 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5941 Data size: 1550601 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -996,10 +996,10 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 1618 Data size: 140766 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1582 Data size: 137634 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1046,10 +1046,10 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 5
-                        Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 1618 Data size: 140766 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1582 Data size: 137634 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1186,10 +1186,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3
                       input vertices:
                         1 Map 4
-                      Statistics: Num rows: 1618 Data size: 576008 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1582 Data size: 563192 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 1618 Data size: 576008 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1582 Data size: 563192 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1214,10 +1214,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3
                       input vertices:
                         1 Map 4
-                      Statistics: Num rows: 1618 Data size: 576008 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1582 Data size: 563192 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 1618 Data size: 576008 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1582 Data size: 563192 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1330,10 +1330,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       input vertices:
                         0 Union 2
-                      Statistics: Num rows: 1618 Data size: 281532 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1582 Data size: 275268 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 1618 Data size: 281532 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1582 Data size: 275268 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/tez_union2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_union2.q.out b/ql/src/test/results/clientpositive/llap/tez_union2.q.out
index 9db0a09..7cbee12 100644
--- a/ql/src/test/results/clientpositive/llap/tez_union2.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_union2.q.out
@@ -61,12 +61,12 @@ STAGE PLANS:
                       keys: _col0 (type: string), _col1 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 408 Data size: 72624 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: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -118,10 +118,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -142,21 +142,21 @@ STAGE PLANS:
                     keys: _col0 (type: string)
                     mode: complete
                     outputColumnNames: _col0
-                    Statistics: Num rows: 309 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 27492 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: _col0 (type: string), _col0 (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 309 Data size: 53766 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 316 Data size: 54984 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: string), _col1 (type: string)
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 72624 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: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 408 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
         Union 2 
             Vertex: Union 2
         Union 5 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out b/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
index 711c5ae..aa9d790 100644
--- a/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
@@ -188,14 +188,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29664 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30336 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), UDFToString(_col1) (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 84048 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 85952 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 309 Data size: 84048 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 85952 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -204,7 +204,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: key, value
-                    Statistics: Num rows: 309 Data size: 84048 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 85952 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                       mode: hash
@@ -1129,14 +1129,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29664 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30336 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), UDFToString(_col1) (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 84048 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 85952 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 309 Data size: 84048 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 85952 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1145,7 +1145,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: key, value
-                    Statistics: Num rows: 309 Data size: 84048 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 85952 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                       mode: hash
@@ -2100,14 +2100,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29664 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30336 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), UDFToString(_col1) (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 84048 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 85952 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 309 Data size: 84048 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 85952 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -2116,7 +2116,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: key, value
-                    Statistics: Num rows: 309 Data size: 84048 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 85952 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                       mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_2.q.out
----------------------------------------------------------------------
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 4ccfe1a..3373fc25 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
@@ -105,11 +105,11 @@ STAGE PLANS:
                   1 UDFToInteger(_col0) (type: int)
                   2 (UDFToInteger(_col0) + 0) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 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 3 
             Execution mode: vectorized, llap
@@ -117,10 +117,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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -296,11 +296,11 @@ STAGE PLANS:
                   1 UDFToInteger(_col0) (type: int)
                   2 (UDFToInteger(_col0) + 0) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 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 3 
             Execution mode: vectorized, llap
@@ -308,10 +308,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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -487,11 +487,11 @@ STAGE PLANS:
                   1 UDFToInteger(_col0) (type: int)
                   2 (UDFToInteger(_col0) + 0) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 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 3 
             Execution mode: vectorized, llap
@@ -499,10 +499,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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/unionDistinct_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/unionDistinct_1.q.out b/ql/src/test/results/clientpositive/llap/unionDistinct_1.q.out
index 8d25098..d6f24fc 100644
--- a/ql/src/test/results/clientpositive/llap/unionDistinct_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/unionDistinct_1.q.out
@@ -3790,13 +3790,13 @@ STAGE PLANS:
                         keys: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3
-                        Statistics: Num rows: 332 Data size: 104486 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 332 Data size: 104130 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                           null sort order: aaaa
                           sort order: ++++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
-                          Statistics: Num rows: 332 Data size: 104486 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 332 Data size: 104130 Basic stats: COMPLETE Column stats: COMPLETE
                           tag: -1
                           auto parallelism: true
             Execution mode: vectorized
@@ -4021,14 +4021,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string), KEY._col3 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 332 Data size: 104486 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 332 Data size: 104130 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   GlobalTableId: 1
                   directory: hdfs://### HDFS PATH ###
                   NumFilesPerFileSink: 1
                   Static Partition Specification: ds=2/
-                  Statistics: Num rows: 332 Data size: 104486 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 332 Data size: 104130 Basic stats: COMPLETE Column stats: COMPLETE
                   Stats Publishing Key Prefix: hdfs://### HDFS PATH ###
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -4057,7 +4057,7 @@ STAGE PLANS:
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), '2' (type: string)
                   outputColumnNames: k1, k2, k3, k4, ds
-                  Statistics: Num rows: 332 Data size: 132706 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 332 Data size: 132350 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: compute_stats(k1, 'hll'), compute_stats(k2, 'hll'), compute_stats(k3, 'hll'), compute_stats(k4, 'hll')
                     keys: ds (type: string)
@@ -4125,22 +4125,22 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col4, _col5
                 Position of Big Table: 0
-                Statistics: Num rows: 166 Data size: 45390 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 166 Data size: 45034 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 166 Data size: 45390 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 166 Data size: 45034 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     keys: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 332 Data size: 104486 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 332 Data size: 104130 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                       null sort order: aaaa
                       sort order: ++++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
-                      Statistics: Num rows: 332 Data size: 104486 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 332 Data size: 104130 Basic stats: COMPLETE Column stats: COMPLETE
                       tag: -1
                       auto parallelism: true
         Union 2 
@@ -6630,12 +6630,12 @@ STAGE PLANS:
                       keys: _col0 (type: string), _col1 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 408 Data size: 72624 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: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized
         Map 4 
             Map Operator Tree:
@@ -6684,20 +6684,20 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 408 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: complete
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col2 (type: bigint), _col0 (type: string), _col1 (type: string)
                     outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
-                      Statistics: Num rows: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 408 Data size: 75888 Basic stats: COMPLETE Column stats: COMPLETE
                       table:
                           input format: org.apache.hadoop.mapred.TextInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -6719,21 +6719,21 @@ STAGE PLANS:
                     keys: _col0 (type: string)
                     mode: complete
                     outputColumnNames: _col0
-                    Statistics: Num rows: 309 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 27492 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: _col0 (type: string), _col0 (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 309 Data size: 53766 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 316 Data size: 54984 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: string), _col1 (type: string)
                         mode: hash
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 408 Data size: 72624 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: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 408 Data size: 72624 Basic stats: COMPLETE Column stats: COMPLETE
         Union 2 
             Vertex: Union 2
         Union 5 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/unionDistinct_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/unionDistinct_3.q.out b/ql/src/test/results/clientpositive/llap/unionDistinct_3.q.out
index ecf82b3..4624cc2 100644
--- a/ql/src/test/results/clientpositive/llap/unionDistinct_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/unionDistinct_3.q.out
@@ -1780,10 +1780,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 66 Data size: 35904 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 65 Data size: 35360 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 66 Data size: 35904 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 65 Data size: 35360 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1929,12 +1929,12 @@ STAGE PLANS:
                       keys: _col0 (type: string)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 309 Data size: 56856 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 316 Data size: 58144 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: 309 Data size: 56856 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 10 
@@ -1950,12 +1950,12 @@ STAGE PLANS:
                       keys: _col0 (type: string)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 309 Data size: 56856 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 316 Data size: 58144 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: 309 Data size: 56856 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 11 
@@ -1971,12 +1971,12 @@ STAGE PLANS:
                       keys: _col0 (type: string)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 309 Data size: 56856 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 316 Data size: 58144 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: 309 Data size: 56856 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 12 
@@ -1992,12 +1992,12 @@ STAGE PLANS:
                       keys: _col0 (type: string)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 160 Data size: 29440 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 163 Data size: 29992 Basic stats: COMPLETE Column stats: PARTIAL
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 160 Data size: 29440 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 163 Data size: 29992 Basic stats: COMPLETE Column stats: PARTIAL
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 13 
@@ -2013,12 +2013,12 @@ STAGE PLANS:
                       keys: _col0 (type: string)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 85 Data size: 15640 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 87 Data size: 16008 Basic stats: COMPLETE Column stats: PARTIAL
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 85 Data size: 15640 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 87 Data size: 16008 Basic stats: COMPLETE Column stats: PARTIAL
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -2028,17 +2028,17 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 309 Data size: 56856 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0
-                  Statistics: Num rows: 309 Data size: 56856 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 58144 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: 309 Data size: 56856 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -2046,17 +2046,17 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 309 Data size: 56856 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 58144 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0
-                  Statistics: Num rows: 160 Data size: 29440 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 163 Data size: 29992 Basic stats: COMPLETE Column stats: PARTIAL
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 160 Data size: 29440 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 163 Data size: 29992 Basic stats: COMPLETE Column stats: PARTIAL
         Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -2064,17 +2064,17 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 160 Data size: 29440 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 163 Data size: 29992 Basic stats: COMPLETE Column stats: PARTIAL
                 Group By Operator
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0
-                  Statistics: Num rows: 85 Data size: 15640 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 87 Data size: 16008 Basic stats: COMPLETE Column stats: PARTIAL
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
-                    Statistics: Num rows: 85 Data size: 15640 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 87 Data size: 16008 Basic stats: COMPLETE Column stats: PARTIAL
         Reducer 9 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -2082,16 +2082,16 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 85 Data size: 15640 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 87 Data size: 16008 Basic stats: COMPLETE Column stats: PARTIAL
                 Group By Operator
                   aggregations: count(1)
                   keys: _col0 (type: string)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 85 Data size: 16320 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 87 Data size: 16704 Basic stats: COMPLETE Column stats: PARTIAL
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 85 Data size: 16320 Basic stats: COMPLETE Column stats: PARTIAL
+                    Statistics: Num rows: 87 Data size: 16704 Basic stats: COMPLETE Column stats: PARTIAL
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/union_fast_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/union_fast_stats.q.out b/ql/src/test/results/clientpositive/llap/union_fast_stats.q.out
index 4f80cbf..66b6bd8 100644
--- a/ql/src/test/results/clientpositive/llap/union_fast_stats.q.out
+++ b/ql/src/test/results/clientpositive/llap/union_fast_stats.q.out
@@ -179,7 +179,7 @@ Table Parameters:
 	bucketing_version   	2                   
 	numFiles            	3                   
 	numRows             	5                   
-	rawDataSize         	1069                
+	rawDataSize         	1300                
 	totalSize           	4033                
 #### A masked pattern was here ####
 	 	 
@@ -513,7 +513,7 @@ Table Parameters:
 	numFiles            	1                   
 	numRows             	5                   
 	rawDataSize         	1069                
-	totalSize           	3245                
+	totalSize           	3247                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -566,7 +566,7 @@ Table Parameters:
 	numFiles            	1                   
 	numRows             	15                  
 	rawDataSize         	3320                
-	totalSize           	3245                
+	totalSize           	3247                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -631,7 +631,7 @@ Table Parameters:
 	numFiles            	2                   
 	numRows             	20                  
 	rawDataSize         	4389                
-	totalSize           	4618                
+	totalSize           	4620                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/union_top_level.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/union_top_level.q.out b/ql/src/test/results/clientpositive/llap/union_top_level.q.out
index f2bc8c8..0085342 100644
--- a/ql/src/test/results/clientpositive/llap/union_top_level.q.out
+++ b/ql/src/test/results/clientpositive/llap/union_top_level.q.out
@@ -274,15 +274,15 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col2 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
-                    Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.1
                     value expressions: _col1 (type: string)
         Reducer 3 
@@ -291,7 +291,7 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 10
                   Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
@@ -312,15 +312,15 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col2 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
-                    Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.1
                     value expressions: _col1 (type: string)
         Reducer 6 
@@ -329,7 +329,7 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 10
                   Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vector_cast_constant.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_cast_constant.q.out b/ql/src/test/results/clientpositive/llap/vector_cast_constant.q.out
index 4c2c743..bbaa05c 100644
--- a/ql/src/test/results/clientpositive/llap/vector_cast_constant.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_cast_constant.q.out
@@ -154,7 +154,7 @@ STAGE PLANS:
                       keys: _col0 (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                      Statistics: Num rows: 256 Data size: 39936 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 257 Data size: 40092 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
@@ -163,7 +163,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: 256 Data size: 39936 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 257 Data size: 40092 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col1 (type: bigint), _col2 (type: bigint), _col3 (type: double), _col4 (type: bigint), _col5 (type: decimal(12,0)), _col6 (type: bigint)
             Execution mode: vectorized, llap
@@ -199,7 +199,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                Statistics: Num rows: 256 Data size: 39936 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 257 Data size: 40092 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), (_col1 / _col2) (type: double), (_col3 / _col4) (type: double), CAST( (_col5 / _col6) AS decimal(6,4)) (type: decimal(6,4))
                   outputColumnNames: _col0, _col1, _col2, _col3
@@ -208,7 +208,7 @@ STAGE PLANS:
                       native: true
                       projectedOutputColumnNums: [0, 7, 8, 11]
                       selectExpressions: LongColDivideLongColumn(col 1:bigint, col 2:bigint) -> 7:double, DoubleColDivideLongColumn(col 3:double, col 4:bigint) -> 8:double, CastDecimalToDecimal(col 10:decimal(32,20))(children: DecimalColDivideDecimalColumn(col 5:decimal(12,0), col 9:decimal(19,0))(children: CastLongToDecimal(col 6:bigint) -> 9:decimal(19,0)) -> 10:decimal(32,20)) -> 11:decimal(6,4)
-                  Statistics: Num rows: 256 Data size: 33792 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 257 Data size: 33924 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
                     sort order: +
@@ -216,7 +216,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: 256 Data size: 33792 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 257 Data size: 33924 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.1
                     value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: decimal(6,4))
         Reducer 3 
@@ -235,7 +235,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3]
-                Statistics: Num rows: 256 Data size: 33792 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 257 Data size: 33924 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 10
                   Limit Vectorization:

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vector_count_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_count_distinct.q.out b/ql/src/test/results/clientpositive/llap/vector_count_distinct.q.out
index 90086ea..42fcdbd 100644
--- a/ql/src/test/results/clientpositive/llap/vector_count_distinct.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_count_distinct.q.out
@@ -1274,7 +1274,7 @@ STAGE PLANS:
                       keys: ws_order_number (type: int)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 169 Data size: 676 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 170 Data size: 680 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
@@ -1283,7 +1283,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: 169 Data size: 676 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 170 Data size: 680 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -1315,7 +1315,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 169 Data size: 676 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 170 Data size: 680 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count(_col0)
                   Group By Vectorization:

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out
index 98e6e54..b99a4ac 100644
--- a/ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out
@@ -155,10 +155,10 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col5
                   input vertices:
                     1 Reducer 6
-                  Statistics: Num rows: 500 Data size: 98620 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 98584 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 98620 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 98584 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: _col0 (type: string), _col1 (type: string)
                       outputColumnNames: _col0, _col1

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out b/ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out
index fccd38a..df9a46e 100644
--- a/ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out
@@ -277,7 +277,7 @@ STAGE PLANS:
                       keys: ss_ticket_number (type: int)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
@@ -286,7 +286,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: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -319,7 +319,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
@@ -327,7 +327,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: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
         Reducer 3 
             Execution mode: vectorized, llap
@@ -345,7 +345,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0]
-                Statistics: Num rows: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Limit Vectorization:
@@ -476,7 +476,7 @@ STAGE PLANS:
                       keys: ss_ticket_number (type: int)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
@@ -485,7 +485,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: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -517,7 +517,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: min(_col0)
                   Group By Vectorization:
@@ -531,7 +531,7 @@ STAGE PLANS:
                   keys: _col0 (type: int)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 82 Data size: 656 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 85 Data size: 680 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col1 (type: int)
                     outputColumnNames: _col0
@@ -539,7 +539,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [1]
-                    Statistics: Num rows: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
                       sort order: +
@@ -547,7 +547,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: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Vectorization:
@@ -564,13 +564,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0]
-                Statistics: Num rows: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 82 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 85 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/vector_left_outer_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_left_outer_join.q.out b/ql/src/test/results/clientpositive/llap/vector_left_outer_join.q.out
index fe3b1e5..6201a48 100644
--- a/ql/src/test/results/clientpositive/llap/vector_left_outer_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_left_outer_join.q.out
@@ -51,7 +51,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 25057 Data size: 87772 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 24737 Data size: 86492 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Left Outer Join 0 to 1
@@ -60,7 +60,7 @@ STAGE PLANS:
                           1 _col0 (type: tinyint)
                         input vertices:
                           1 Map 4
-                        Statistics: Num rows: 2424412 Data size: 19395296 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 2338217 Data size: 18705736 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash


[07/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
----------------------------------------------------------------------
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 c75e82c..95098b9 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
@@ -26,11 +26,11 @@ Stage-0
     Stage-1
       Reducer 3 vectorized, llap
       File Output Operator [FS_25]
-        Select Operator [SEL_24] (rows=2076 width=553)
+        Select Operator [SEL_24] (rows=2048 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_17] (rows=2076 width=553)
+            Merge Join Operator [MERGEJOIN_17] (rows=2048 width=552)
               Conds:RS_20._col2=RS_23._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_20]
@@ -111,7 +111,7 @@ Stage-0
           PARTITION_ONLY_SHUFFLE [RS_11]
             Group By Operator [GBY_10] (rows=1 width=8)
               Output:["_col0"],aggregations:["count()"]
-              Merge Join Operator [MERGEJOIN_19] (rows=2076 width=8)
+              Merge Join Operator [MERGEJOIN_19] (rows=2048 width=8)
                 Conds:RS_22._col0=RS_25._col0(Inner)
               <-Map 1 [SIMPLE_EDGE] vectorized, llap
                 SHUFFLE [RS_22]
@@ -180,18 +180,18 @@ Stage-0
     Stage-1
       Reducer 4 vectorized, llap
       File Output Operator [FS_31]
-        Select Operator [SEL_30] (rows=623 width=11)
+        Select Operator [SEL_30] (rows=631 width=11)
           Output:["_col0","_col1"]
         <-Reducer 3 [SIMPLE_EDGE] vectorized, llap
           SHUFFLE [RS_29]
-            Group By Operator [GBY_28] (rows=623 width=11)
+            Group By Operator [GBY_28] (rows=631 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=623 width=11)
+                Group By Operator [GBY_10] (rows=631 width=11)
                   Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                  Merge Join Operator [MERGEJOIN_21] (rows=2076 width=3)
+                  Merge Join Operator [MERGEJOIN_21] (rows=2048 width=3)
                     Conds:RS_24._col1=RS_27._col0(Inner),Output:["_col0"]
                   <-Map 1 [SIMPLE_EDGE] vectorized, llap
                     SHUFFLE [RS_24]

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/groupby1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/groupby1.q.out b/ql/src/test/results/clientpositive/llap/groupby1.q.out
index a678d27..e1cc298 100644
--- a/ql/src/test/results/clientpositive/llap/groupby1.q.out
+++ b/ql/src/test/results/clientpositive/llap/groupby1.q.out
@@ -67,14 +67,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: UDFToInteger(_col0) (type: int), _col1 (type: double)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 3708 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 3792 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 309 Data size: 3708 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 3792 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/groupby2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/groupby2.q.out b/ql/src/test/results/clientpositive/llap/groupby2.q.out
index 4499ee7..434be17 100644
--- a/ql/src/test/results/clientpositive/llap/groupby2.q.out
+++ b/ql/src/test/results/clientpositive/llap/groupby2.q.out
@@ -70,14 +70,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 309 Data size: 61800 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 63200 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 309 Data size: 114948 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 309 Data size: 114948 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -86,11 +86,11 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string)
                     outputColumnNames: key, c1, c2
-                    Statistics: Num rows: 309 Data size: 114948 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       sort order: 
                       Map-reduce partition columns: rand() (type: double)
-                      Statistics: Num rows: 309 Data size: 114948 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 316 Data size: 117552 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: key (type: string), c1 (type: int), c2 (type: string)
         Reducer 4 
             Execution mode: llap

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/groupby_resolution.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/groupby_resolution.q.out b/ql/src/test/results/clientpositive/llap/groupby_resolution.q.out
index bb6bec2..39dd4d5 100644
--- a/ql/src/test/results/clientpositive/llap/groupby_resolution.q.out
+++ b/ql/src/test/results/clientpositive/llap/groupby_resolution.q.out
@@ -38,10 +38,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: complete
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -93,10 +93,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: complete
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -164,10 +164,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -235,10 +235,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
index 923b8fb..e98f3ba 100644
--- a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
@@ -56,7 +56,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 25057 Data size: 200456 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash
@@ -177,7 +177,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 25057 Data size: 200456 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash
@@ -296,7 +296,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 18702 Data size: 149616 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 18464 Data size: 147712 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash
@@ -413,7 +413,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 18702 Data size: 149616 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 18464 Data size: 147712 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash
@@ -527,7 +527,7 @@ STAGE PLANS:
                         1 _col0 (type: int)
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 25057 Data size: 200456 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -638,7 +638,7 @@ STAGE PLANS:
                         1 _col0 (type: int)
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 25057 Data size: 200456 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
index 67555c0..ed7d8c9 100644
--- a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
@@ -73,7 +73,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 4
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -204,7 +204,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 4
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -349,7 +349,7 @@ STAGE PLANS:
                         0 Map 1
                         2 Map 4
                         3 Map 5
-                      Statistics: Num rows: 1694 Data size: 13552 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1584 Data size: 12672 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -502,7 +502,7 @@ STAGE PLANS:
                         0 Map 1
                         2 Map 4
                         3 Map 5
-                      Statistics: Num rows: 1694 Data size: 13552 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1584 Data size: 12672 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -685,7 +685,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 6
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -746,7 +746,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 7
                         2 Map 10
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 265 Data size: 2120 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -935,7 +935,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 6
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -996,7 +996,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 7
                         2 Map 10
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 265 Data size: 2120 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -1180,7 +1180,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 4
-                      Statistics: Num rows: 20 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 19 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1192,7 +1192,7 @@ STAGE PLANS:
                         input vertices:
                           1 Map 5
                           2 Map 6
-                        Statistics: Num rows: 204 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 196 Data size: 1568 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash
@@ -1375,7 +1375,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 4
-                      Statistics: Num rows: 20 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 19 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1387,7 +1387,7 @@ STAGE PLANS:
                         input vertices:
                           1 Map 5
                           2 Map 6
-                        Statistics: Num rows: 204 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 196 Data size: 1568 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/join1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/join1.q.out b/ql/src/test/results/clientpositive/llap/join1.q.out
index 1aabf2f..cf14327 100644
--- a/ql/src/test/results/clientpositive/llap/join1.q.out
+++ b/ql/src/test/results/clientpositive/llap/join1.q.out
@@ -78,14 +78,14 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col2
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: UDFToInteger(_col0) (type: int), _col2 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -94,7 +94,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: int), _col1 (type: string)
                     outputColumnNames: key, value
-                    Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                       mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/join32_lessSize.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/join32_lessSize.q.out b/ql/src/test/results/clientpositive/llap/join32_lessSize.q.out
index 8445bdc..e4b5673 100644
--- a/ql/src/test/results/clientpositive/llap/join32_lessSize.q.out
+++ b/ql/src/test/results/clientpositive/llap/join32_lessSize.q.out
@@ -67,13 +67,13 @@ STAGE PLANS:
                         input vertices:
                           1 Map 4
                         Position of Big Table: 0
-                        Statistics: Num rows: 40 Data size: 10640 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 10374 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col3 (type: string)
                           null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col3 (type: string)
-                          Statistics: Num rows: 40 Data size: 10640 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 39 Data size: 10374 Basic stats: COMPLETE Column stats: COMPLETE
                           tag: 0
                           value expressions: _col1 (type: string), _col2 (type: string)
                           auto parallelism: true
@@ -300,17 +300,17 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2, _col4
                 Position of Big Table: 1
-                Statistics: Num rows: 64 Data size: 17152 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 63 Data size: 16884 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col2 (type: string), _col4 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 64 Data size: 17152 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 63 Data size: 16884 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
                     NumFilesPerFileSink: 1
-                    Statistics: Num rows: 64 Data size: 17152 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 63 Data size: 16884 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
@@ -341,7 +341,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                     outputColumnNames: key, value, val2
-                    Statistics: Num rows: 64 Data size: 17152 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 63 Data size: 16884 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll'), compute_stats(val2, 'hll')
                       mode: hash
@@ -569,8 +569,7 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 5 (BROADCAST_EDGE)
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
         Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -589,27 +588,15 @@ STAGE PLANS:
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        Estimated key counts: Map 5 => 25
-                        keys:
-                          0 _col0 (type: string)
-                          1 _col0 (type: string)
-                        outputColumnNames: _col0, _col1, _col3
-                        input vertices:
-                          1 Map 5
-                        Position of Big Table: 0
-                        Statistics: Num rows: 39 Data size: 10296 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col1 (type: string)
-                          null sort order: a
-                          sort order: +
-                          Map-reduce partition columns: _col1 (type: string)
-                          Statistics: Num rows: 39 Data size: 10296 Basic stats: COMPLETE Column stats: COMPLETE
-                          tag: 0
-                          value expressions: _col0 (type: string), _col3 (type: string)
-                          auto parallelism: true
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: a
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                        tag: 0
+                        value expressions: _col1 (type: string)
+                        auto parallelism: true
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Path -> Alias:
@@ -746,24 +733,25 @@ STAGE PLANS:
         Map 6 
             Map Operator Tree:
                 TableScan
-                  alias: w
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: y
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: value is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: value (type: string)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
-                        tag: 1
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        tag: 2
+                        value expressions: _col1 (type: string)
                         auto parallelism: true
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -819,29 +807,28 @@ STAGE PLANS:
                     name: default.src
                   name: default.src
             Truncated Path -> Alias:
-              /src [w]
+              /src [y]
         Map 7 
             Map Operator Tree:
                 TableScan
-                  alias: y
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: w
+                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: string), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: value (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: 1
-                        value expressions: _col1 (type: string)
                         auto parallelism: true
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -897,7 +884,7 @@ STAGE PLANS:
                     name: default.src
                   name: default.src
             Truncated Path -> Alias:
-              /src [y]
+              /src [w]
         Reducer 2 
             Execution mode: llap
             Needs Tagging: false
@@ -905,20 +892,22 @@ STAGE PLANS:
               Merge Join Operator
                 condition map:
                      Inner Join 0 to 1
+                     Inner Join 0 to 2
                 keys:
-                  0 _col1 (type: string)
+                  0 _col0 (type: string)
                   1 _col0 (type: string)
-                outputColumnNames: _col0, _col3
-                Position of Big Table: 1
-                Statistics: Num rows: 63 Data size: 11025 Basic stats: COMPLETE Column stats: COMPLETE
+                  2 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col3, _col5
+                Position of Big Table: 2
+                Statistics: Num rows: 61 Data size: 21655 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col0 (type: string)
+                  key expressions: _col1 (type: string)
                   null sort order: a
                   sort order: +
-                  Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 63 Data size: 11025 Basic stats: COMPLETE Column stats: COMPLETE
+                  Map-reduce partition columns: _col1 (type: string)
+                  Statistics: Num rows: 61 Data size: 21655 Basic stats: COMPLETE Column stats: COMPLETE
                   tag: 0
-                  value expressions: _col3 (type: string)
+                  value expressions: _col0 (type: string), _col3 (type: string), _col5 (type: string)
                   auto parallelism: true
         Reducer 3 
             Execution mode: llap
@@ -928,21 +917,21 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: string)
+                  0 _col1 (type: string)
                   1 _col0 (type: string)
-                outputColumnNames: _col0, _col3, _col6
+                outputColumnNames: _col0, _col3, _col5
                 Position of Big Table: 0
-                Statistics: Num rows: 101 Data size: 26866 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 99 Data size: 26334 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: string), _col3 (type: string), _col6 (type: string)
+                  expressions: _col0 (type: string), _col3 (type: string), _col5 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 101 Data size: 26866 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 99 Data size: 26334 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
                     NumFilesPerFileSink: 1
-                    Statistics: Num rows: 101 Data size: 26866 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 99 Data size: 26334 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
@@ -973,7 +962,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                     outputColumnNames: key, value, val2
-                    Statistics: Num rows: 101 Data size: 26866 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 99 Data size: 26334 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll'), compute_stats(val2, 'hll')
                       mode: hash
@@ -1305,17 +1294,17 @@ STAGE PLANS:
                         input vertices:
                           1 Map 5
                         Position of Big Table: 0
-                        Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: string), _col2 (type: string)
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col1 (type: string)
                             null sort order: a
                             sort order: +
                             Map-reduce partition columns: _col1 (type: string)
-                            Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                             tag: 1
                             value expressions: _col0 (type: string)
                             auto parallelism: true
@@ -1464,17 +1453,17 @@ STAGE PLANS:
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col3, _col4
                 Position of Big Table: 0
-                Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col3 (type: string), _col0 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
                     NumFilesPerFileSink: 1
-                    Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
@@ -1505,7 +1494,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                     outputColumnNames: key, value, val2
-                    Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll'), compute_stats(val2, 'hll')
                       mode: hash
@@ -1977,17 +1966,17 @@ STAGE PLANS:
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col3, _col4
                 Position of Big Table: 0
-                Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col3 (type: string), _col0 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
                     NumFilesPerFileSink: 1
-                    Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
@@ -2018,7 +2007,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                     outputColumnNames: key, value, val2
-                    Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll'), compute_stats(val2, 'hll')
                       mode: hash
@@ -2074,17 +2063,17 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
                 Position of Big Table: 1
-                Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string), _col2 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col1 (type: string)
                     null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col1 (type: string)
-                    Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                     tag: 1
                     value expressions: _col0 (type: string)
                     auto parallelism: true
@@ -2327,16 +2316,16 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2
                         input vertices:
                           1 Map 5
-                        Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: string), _col2 (type: string)
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col1 (type: string)
                             sort order: +
                             Map-reduce partition columns: _col1 (type: string)
-                            Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -2370,14 +2359,14 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col3, _col4
-                Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col3 (type: string), _col0 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -2386,7 +2375,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                     outputColumnNames: key, value, val2
-                    Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll'), compute_stats(val2, 'hll')
                       mode: hash
@@ -2617,16 +2606,16 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2
                         input vertices:
                           1 Map 5
-                        Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: string), _col2 (type: string)
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col1 (type: string)
                             sort order: +
                             Map-reduce partition columns: _col1 (type: string)
-                            Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -2660,14 +2649,14 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col3, _col4
-                Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col3 (type: string), _col0 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -2676,7 +2665,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                     outputColumnNames: key, value, val2
-                    Statistics: Num rows: 64 Data size: 17024 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 63 Data size: 16758 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll'), compute_stats(val2, 'hll')
                       mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/join_max_hashtable.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/join_max_hashtable.q.out b/ql/src/test/results/clientpositive/llap/join_max_hashtable.q.out
index c343154..996b789 100644
--- a/ql/src/test/results/clientpositive/llap/join_max_hashtable.q.out
+++ b/ql/src/test/results/clientpositive/llap/join_max_hashtable.q.out
@@ -39,10 +39,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -116,10 +116,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -223,11 +223,11 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 input vertices:
                   1 Map 3
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 HybridGraceHashJoin: true
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -309,11 +309,11 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 input vertices:
                   1 Map 3
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 HybridGraceHashJoin: true
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -396,11 +396,11 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 input vertices:
                   1 Map 3
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 HybridGraceHashJoin: true
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -482,11 +482,11 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 input vertices:
                   1 Map 3
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 HybridGraceHashJoin: true
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -539,10 +539,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -616,10 +616,10 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out b/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out
index 4db4f12..76a8bf6 100644
--- a/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out
+++ b/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out
@@ -66,7 +66,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 1
                   Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
@@ -167,7 +167,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 1
                   Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
@@ -930,7 +930,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 1
                   Offset of rows: 1
@@ -1032,7 +1032,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 288004 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 1
                   Offset of rows: 1

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out b/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out
index 76fae9a..fe8b98f 100644
--- a/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out
@@ -399,12 +399,12 @@ STAGE PLANS:
                       keys: cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: double)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: double)
-                        Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.3
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -415,7 +415,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Statistics: Num rows: 20 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
@@ -513,7 +513,7 @@ STAGE PLANS:
                   keys: _col0 (type: tinyint)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                   Limit
                     Number of rows: 20
                     Statistics: Num rows: 20 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
@@ -611,7 +611,7 @@ STAGE PLANS:
                   keys: _col0 (type: tinyint)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                   Limit
                     Number of rows: 20
                     Statistics: Num rows: 20 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
@@ -706,7 +706,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 128 Data size: 2436 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 2492 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Statistics: Num rows: 20 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1074,7 +1074,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: complete
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 30591 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 307 Data size: 30393 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Statistics: Num rows: 20 Data size: 1980 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/limit_pushdown3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/limit_pushdown3.q.out b/ql/src/test/results/clientpositive/llap/limit_pushdown3.q.out
index a41091c..260ce3d 100644
--- a/ql/src/test/results/clientpositive/llap/limit_pushdown3.q.out
+++ b/ql/src/test/results/clientpositive/llap/limit_pushdown3.q.out
@@ -428,12 +428,12 @@ STAGE PLANS:
                       keys: cdouble (type: double)
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: double)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: double)
-                        Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.3
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -444,11 +444,11 @@ STAGE PLANS:
                 keys: KEY._col0 (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: double)
                   sort order: +
-                  Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.3
         Reducer 3 
             Execution mode: vectorized, llap
@@ -456,7 +456,7 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: double)
                 outputColumnNames: _col0
-                Statistics: Num rows: 5567 Data size: 33256 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 5528 Data size: 33024 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Statistics: Num rows: 20 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
@@ -555,11 +555,11 @@ STAGE PLANS:
                   keys: _col0 (type: tinyint)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: tinyint)
                     sort order: +
-                    Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.3
                     value expressions: _col1 (type: bigint)
         Reducer 3 
@@ -568,7 +568,7 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: tinyint), VALUE._col0 (type: bigint)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Statistics: Num rows: 20 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
@@ -667,11 +667,11 @@ STAGE PLANS:
                   keys: _col0 (type: tinyint)
                   mode: complete
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: tinyint)
                     sort order: +
-                    Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.3
                     value expressions: _col1 (type: bigint)
         Reducer 3 
@@ -680,7 +680,7 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: tinyint), VALUE._col0 (type: bigint)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 128 Data size: 1412 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 1444 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Statistics: Num rows: 20 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
@@ -776,11 +776,11 @@ STAGE PLANS:
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 128 Data size: 2436 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 2492 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: tinyint)
                   sort order: +
-                  Statistics: Num rows: 128 Data size: 2436 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 2492 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.3
                   value expressions: _col1 (type: bigint), _col2 (type: bigint)
         Reducer 3 
@@ -789,7 +789,7 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: tinyint), VALUE._col0 (type: bigint), VALUE._col1 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 128 Data size: 2436 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 2492 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Statistics: Num rows: 20 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1013,11 +1013,11 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: complete
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 30591 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 307 Data size: 30393 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
-                  Statistics: Num rows: 309 Data size: 30591 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 307 Data size: 30393 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.3
                   value expressions: _col1 (type: double)
         Reducer 3 
@@ -1026,7 +1026,7 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: double)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 30591 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 307 Data size: 30393 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 20
                   Statistics: Num rows: 20 Data size: 1980 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out b/ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out
index 6402848..b361b1e 100644
--- a/ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out
+++ b/ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out
@@ -65,7 +65,7 @@ STAGE PLANS:
                             nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Fast Hash Table and No Hybrid Hash Join IS true
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 25057 Data size: 200456 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           Group By Vectorization:
@@ -244,7 +244,7 @@ STAGE PLANS:
                             nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Fast Hash Table and No Hybrid Hash Join IS true
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 25057 Data size: 200456 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           Group By Vectorization:


[02/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out
index 68aabb0..4a10953 100644
--- a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out
@@ -193,48 +193,48 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: string)
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Map 4 
             Map Operator Tree:
                 TableScan
-                  alias: c
+                  alias: b
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Map 5 
             Map Operator Tree:
                 TableScan
-                  alias: b
+                  alias: c
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: value is not null (type: boolean)
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col1 (type: string)
+                        key expressions: _col0 (type: int)
                         sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
@@ -242,28 +242,28 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col1
+                  0 _col1 (type: string)
+                  1 _col1 (type: string)
+                outputColumnNames: _col0, _col2
                 Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col1 (type: string)
+                  key expressions: _col0 (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col1 (type: string)
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: int)
+                  value expressions: _col2 (type: int)
         Reducer 3 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
-                  1 _col1 (type: string)
-                outputColumnNames: _col0, _col3
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col2
                 Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: int), _col3 (type: int)
+                  expressions: _col0 (type: int), _col2 (type: int)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
@@ -309,48 +309,48 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: string)
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Map 4 
             Map Operator Tree:
                 TableScan
-                  alias: c
+                  alias: b
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Map 5 
             Map Operator Tree:
                 TableScan
-                  alias: b
+                  alias: c
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: value is not null (type: boolean)
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col1 (type: string)
+                        key expressions: _col0 (type: int)
                         sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
@@ -358,28 +358,28 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col1
+                  0 _col1 (type: string)
+                  1 _col1 (type: string)
+                outputColumnNames: _col0, _col2
                 Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col1 (type: string)
+                  key expressions: _col0 (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col1 (type: string)
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: int)
+                  value expressions: _col2 (type: int)
         Reducer 3 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
-                  1 _col1 (type: string)
-                outputColumnNames: _col0, _col3
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col2
                 Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: int), _col3 (type: int)
+                  expressions: _col0 (type: int), _col2 (type: int)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
@@ -1906,48 +1906,48 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: string)
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Map 4 
             Map Operator Tree:
                 TableScan
-                  alias: c
+                  alias: b
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Map 5 
             Map Operator Tree:
                 TableScan
-                  alias: b
+                  alias: c
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: value is not null (type: boolean)
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col1 (type: string)
+                        key expressions: _col0 (type: int)
                         sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
@@ -1955,28 +1955,28 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col1
+                  0 _col1 (type: string)
+                  1 _col1 (type: string)
+                outputColumnNames: _col0, _col2
                 Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col1 (type: string)
+                  key expressions: _col0 (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col1 (type: string)
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: int)
+                  value expressions: _col2 (type: int)
         Reducer 3 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
-                  1 _col1 (type: string)
-                outputColumnNames: _col0, _col3
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col2
                 Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: int), _col3 (type: int)
+                  expressions: _col0 (type: int), _col2 (type: int)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
@@ -2023,48 +2023,48 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: string)
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Map 4 
             Map Operator Tree:
                 TableScan
-                  alias: c
+                  alias: b
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
+                        key expressions: _col1 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Map 5 
             Map Operator Tree:
                 TableScan
-                  alias: b
+                  alias: c
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: value is not null (type: boolean)
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col1 (type: string)
+                        key expressions: _col0 (type: int)
                         sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int)
             Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
@@ -2072,28 +2072,28 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col1
+                  0 _col1 (type: string)
+                  1 _col1 (type: string)
+                outputColumnNames: _col0, _col2
                 Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col1 (type: string)
+                  key expressions: _col0 (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col1 (type: string)
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: int)
+                  value expressions: _col2 (type: int)
         Reducer 3 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
-                  1 _col1 (type: string)
-                outputColumnNames: _col0, _col3
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col2
                 Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: int), _col3 (type: int)
+                  expressions: _col0 (type: int), _col2 (type: int)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out b/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
index b1363f0..ddd6bd1 100644
--- a/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
+++ b/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
@@ -488,26 +488,25 @@ JOIN src y ON (x.key = y.key)
 JOIN src1 z ON (x.key = z.key)
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-4 is a root stage
-  Stage-3 depends on stages: Stage-4
+  Stage-3 is a root stage
   Stage-1 depends on stages: Stage-3
   Stage-0 depends on stages: Stage-1
   Stage-2 depends on stages: Stage-0
 
 STAGE PLANS:
-  Stage: Stage-4
+  Stage: Stage-3
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 2 
+        Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: z
+                  alias: x
                   Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: key is not null (type: boolean)
+                    predicate: (key is not null and value is not null) (type: boolean)
                     Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key (type: string), value (type: string)
@@ -517,7 +516,8 @@ STAGE PLANS:
                         keys:
                           0 _col0 (type: string)
                           1 _col0 (type: string)
-                        Position of Big Table: 0
+                          2 _col0 (type: string)
+                        Position of Big Table: 2
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
@@ -573,42 +573,27 @@ STAGE PLANS:
                     name: default.src1
                   name: default.src1
             Truncated Path -> Alias:
-              /src1 [$hdt$_3:z]
-
-  Stage: Stage-3
-    Spark
-#### A masked pattern was here ####
-      Vertices:
-        Map 1 
+              /src1 [$hdt$_2:x]
+        Map 2 
             Map Operator Tree:
                 TableScan
-                  alias: x
+                  alias: z
                   Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: (key is not null and value is not null) (type: boolean)
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
+                      Spark HashTable Sink Operator
                         keys:
                           0 _col0 (type: string)
                           1 _col0 (type: string)
-                        outputColumnNames: _col0, _col1, _col3
-                        input vertices:
-                          1 Map 2
-                        Position of Big Table: 0
-                        Statistics: Num rows: 27 Data size: 210 Basic stats: COMPLETE Column stats: NONE
-                        Spark HashTable Sink Operator
-                          keys:
-                            0 _col1 (type: string)
-                            1 _col0 (type: string)
-                          Position of Big Table: 1
+                          2 _col0 (type: string)
+                        Position of Big Table: 2
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
@@ -664,24 +649,24 @@ STAGE PLANS:
                     name: default.src1
                   name: default.src1
             Truncated Path -> Alias:
-              /src1 [$hdt$_2:x]
+              /src1 [$hdt$_3:z]
         Map 4 
             Map Operator Tree:
                 TableScan
-                  alias: y
+                  alias: w
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: key is not null (type: boolean)
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: string), value (type: string)
-                      outputColumnNames: _col0, _col1
+                      expressions: value (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Spark HashTable Sink Operator
                         keys:
-                          0 _col0 (type: string)
+                          0 _col1 (type: string)
                           1 _col0 (type: string)
                         Position of Big Table: 0
             Execution mode: vectorized
@@ -739,7 +724,7 @@ STAGE PLANS:
                     name: default.src
                   name: default.src
             Truncated Path -> Alias:
-              /src [$hdt$_0:y]
+              /src [$hdt$_0:w]
 
   Stage: Stage-1
     Spark
@@ -748,49 +733,52 @@ STAGE PLANS:
         Map 3 
             Map Operator Tree:
                 TableScan
-                  alias: w
+                  alias: y
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: value is not null (type: boolean)
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: value (type: string)
-                      outputColumnNames: _col0
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
+                             Inner Join 0 to 2
                         keys:
-                          0 _col1 (type: string)
+                          0 _col0 (type: string)
                           1 _col0 (type: string)
-                        outputColumnNames: _col0, _col3
+                          2 _col0 (type: string)
+                        outputColumnNames: _col0, _col1, _col3, _col5
                         input vertices:
                           0 Map 1
-                        Position of Big Table: 1
-                        Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                          1 Map 2
+                        Position of Big Table: 2
+                        Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
                           keys:
-                            0 _col0 (type: string)
+                            0 _col1 (type: string)
                             1 _col0 (type: string)
-                          outputColumnNames: _col0, _col3, _col6
+                          outputColumnNames: _col0, _col3, _col5
                           input vertices:
                             1 Map 4
                           Position of Big Table: 0
-                          Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE
                           Select Operator
-                            expressions: _col0 (type: string), _col3 (type: string), _col6 (type: string)
+                            expressions: _col0 (type: string), _col3 (type: string), _col5 (type: string)
                             outputColumnNames: _col0, _col1, _col2
-                            Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+                            Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE
                             File Output Operator
                               compressed: false
                               GlobalTableId: 1
 #### A masked pattern was here ####
                               NumFilesPerFileSink: 1
-                              Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+                              Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE
 #### A masked pattern was here ####
                               table:
                                   input format: org.apache.hadoop.mapred.TextInputFormat
@@ -873,7 +861,7 @@ STAGE PLANS:
                     name: default.src
                   name: default.src
             Truncated Path -> Alias:
-              /src [$hdt$_1:w]
+              /src [$hdt$_1:y]
 
   Stage: Stage-0
     Move Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out b/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
index ec632a6..fff2f31 100644
--- a/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
@@ -566,18 +566,18 @@ STAGE PLANS:
         Map 5 
             Map Operator Tree:
                 TableScan
-                  alias: src1
+                  alias: src
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: key (type: string)
+                      expressions: value (type: string)
                       outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Spark HashTable Sink Operator
                         keys:
-                          0 _col0 (type: string)
+                          0 _col1 (type: string)
                           1 _col0 (type: string)
             Execution mode: vectorized
             Local Work:
@@ -603,22 +603,22 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col1 (type: string)
+                        key expressions: _col0 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                        Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: string), _col2 (type: string)
+                        value expressions: _col1 (type: string), _col2 (type: string)
             Execution mode: vectorized
         Map 4 
             Map Operator Tree:
                 TableScan
-                  alias: src
+                  alias: src1
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: value is not null (type: boolean)
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: value (type: string)
+                      expressions: key (type: string)
                       outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
@@ -635,15 +635,15 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
+                  0 _col0 (type: string)
                   1 _col0 (type: string)
-                outputColumnNames: _col0, _col2
+                outputColumnNames: _col1, _col2
                 Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE
                 Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: string)
+                    0 _col1 (type: string)
                     1 _col0 (type: string)
                   outputColumnNames: _col2
                   input vertices:

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/spark/spark_explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/spark_explainuser_1.q.out b/ql/src/test/results/clientpositive/spark/spark_explainuser_1.q.out
index 103491d..85d0b8a 100644
--- a/ql/src/test/results/clientpositive/spark/spark_explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/spark/spark_explainuser_1.q.out
@@ -1665,9 +1665,9 @@ Stage-0
     Stage-1
       Reducer 2
       File Output Operator [FS_19]
-        Select Operator [SEL_18] (rows=366 width=178)
+        Select Operator [SEL_18] (rows=365 width=178)
           Output:["_col0","_col1"]
-          Filter Operator [FIL_17] (rows=366 width=179)
+          Filter Operator [FIL_17] (rows=365 width=179)
             predicate:_col3 is null
             Join Operator [JOIN_16] (rows=500 width=179)
               Output:["_col0","_col1","_col3"],condition map:[{"":"{\"type\":\"Left Outer\",\"left\":0,\"right\":1}"}],keys:{"0":"_col1","1":"_col0"}
@@ -1730,9 +1730,9 @@ Stage-0
     Stage-1
       Reducer 3
       File Output Operator [FS_18]
-        Select Operator [SEL_17] (rows=183 width=178)
+        Select Operator [SEL_17] (rows=185 width=178)
           Output:["_col0","_col1"]
-          Filter Operator [FIL_16] (rows=183 width=179)
+          Filter Operator [FIL_16] (rows=185 width=179)
             predicate:_col4 is null
             Join Operator [JOIN_15] (rows=250 width=179)
               Output:["_col0","_col1","_col4"],condition map:[{"":"{\"type\":\"Left Outer\",\"left\":0,\"right\":1}"}],keys:{"0":"_col0, _col1","1":"_col0, _col1"}
@@ -1806,7 +1806,7 @@ Stage-0
     Stage-1
       Reducer 2
       File Output Operator [FS_12]
-        Join Operator [JOIN_10] (rows=133 width=178)
+        Join Operator [JOIN_10] (rows=131 width=178)
           Output:["_col0","_col1"],condition map:[{"":"{\"type\":\"Left Semi\",\"left\":0,\"right\":1}"}],keys:{"0":"_col0, _col1","1":"_col0, _col1"}
         <-Map 1 [PARTITION-LEVEL SORT]
           PARTITION-LEVEL SORT [RS_8]
@@ -1858,7 +1858,7 @@ Stage-0
     Stage-1
       Reducer 2
       File Output Operator [FS_12]
-        Join Operator [JOIN_10] (rows=133 width=178)
+        Join Operator [JOIN_10] (rows=131 width=178)
           Output:["_col0","_col1"],condition map:[{"":"{\"type\":\"Left Semi\",\"left\":0,\"right\":1}"}],keys:{"0":"_col0, _col1","1":"_col0, _col1"}
         <-Map 1 [PARTITION-LEVEL SORT]
           PARTITION-LEVEL SORT [RS_8]
@@ -1900,7 +1900,7 @@ Stage-0
     Stage-1
       Reducer 2
       File Output Operator [FS_12]
-        Join Operator [JOIN_10] (rows=133 width=178)
+        Join Operator [JOIN_10] (rows=131 width=178)
           Output:["_col0","_col1"],condition map:[{"":"{\"type\":\"Left Semi\",\"left\":0,\"right\":1}"}],keys:{"0":"_col0","1":"_col0"}
         <-Map 1 [PARTITION-LEVEL SORT]
           PARTITION-LEVEL SORT [RS_8]
@@ -2021,16 +2021,16 @@ Stage-0
         <-Reducer 3 [PARTITION-LEVEL SORT]
           PARTITION-LEVEL SORT [RS_27]
             PartitionCols:_col2
-            Filter Operator [FIL_15] (rows=66 width=186)
+            Filter Operator [FIL_15] (rows=65 width=186)
               predicate:_col2 is not null
-              Group By Operator [GBY_14] (rows=66 width=186)
+              Group By Operator [GBY_14] (rows=65 width=186)
                 Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
               <-Reducer 2 [GROUP]
                 GROUP [RS_13]
                   PartitionCols:_col0, _col1
-                  Group By Operator [GBY_12] (rows=66 width=186)
+                  Group By Operator [GBY_12] (rows=65 width=186)
                     Output:["_col0","_col1","_col2"],aggregations:["count()"],keys:_col0, _col1
-                    Join Operator [JOIN_10] (rows=133 width=178)
+                    Join Operator [JOIN_10] (rows=131 width=178)
                       Output:["_col0","_col1"],condition map:[{"":"{\"type\":\"Left Semi\",\"left\":0,\"right\":1}"}],keys:{"0":"_col0","1":"_col0"}
                     <-Map 1 [PARTITION-LEVEL SORT]
                       PARTITION-LEVEL SORT [RS_8]
@@ -2634,7 +2634,7 @@ Stage-0
                     PartitionCols:_col0
                     Group By Operator [GBY_10] (rows=16 width=94)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                      Join Operator [JOIN_8] (rows=40 width=86)
+                      Join Operator [JOIN_8] (rows=39 width=86)
                         Output:["_col0"],condition map:[{"":"{\"type\":\"Inner\",\"left\":0,\"right\":1}"}],keys:{"0":"_col0","1":"_col0"}
                       <-Map 1 [PARTITION-LEVEL SORT]
                         PARTITION-LEVEL SORT [RS_6]
@@ -2695,7 +2695,7 @@ Stage-0
                     PartitionCols:_col0
                     Group By Operator [GBY_10] (rows=16 width=94)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                      Join Operator [JOIN_8] (rows=40 width=86)
+                      Join Operator [JOIN_8] (rows=39 width=86)
                         Output:["_col0"],condition map:[{"":"{\"type\":\"Inner\",\"left\":0,\"right\":1}"}],keys:{"0":"_col0","1":"_col0"}
                       <-Map 1 [PARTITION-LEVEL SORT]
                         PARTITION-LEVEL SORT [RS_6]
@@ -2755,7 +2755,7 @@ Stage-0
                     PartitionCols:_col0
                     Group By Operator [GBY_10] (rows=16 width=94)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                      Map Join Operator [MAPJOIN_22] (rows=40 width=86)
+                      Map Join Operator [MAPJOIN_22] (rows=39 width=86)
                         Conds:SEL_5._col0=SEL_5._col0(Inner),Output:["_col0"]
                       <-Select Operator [SEL_5] (rows=500 width=87)
                           Output:["_col0"]
@@ -2807,16 +2807,16 @@ Stage-0
           GROUP [RS_18]
             Group By Operator [GBY_17] (rows=1 width=16)
               Output:["_col0","_col1"],aggregations:["sum(_col0)","sum(_col1)"]
-              Select Operator [SEL_15] (rows=10 width=94)
+              Select Operator [SEL_15] (rows=9 width=94)
                 Output:["_col0","_col1"]
-                Group By Operator [GBY_14] (rows=10 width=94)
+                Group By Operator [GBY_14] (rows=9 width=94)
                   Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
                 <-Reducer 2 [GROUP]
                   GROUP [RS_13]
                     PartitionCols:_col0
-                    Group By Operator [GBY_12] (rows=10 width=94)
+                    Group By Operator [GBY_12] (rows=9 width=94)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                      Join Operator [JOIN_10] (rows=20 width=86)
+                      Join Operator [JOIN_10] (rows=19 width=86)
                         Output:["_col0"],condition map:[{"":"{\"type\":\"Left Semi\",\"left\":0,\"right\":1}"}],keys:{"0":"_col0","1":"_col0"}
                       <-Map 1 [PARTITION-LEVEL SORT]
                         PARTITION-LEVEL SORT [RS_8]
@@ -5215,9 +5215,9 @@ Stage-2
               Reducer 2
               File Output Operator [FS_11]
                 table:{"name:":"default.dest_j1_n14"}
-                Select Operator [SEL_9] (rows=809 width=95)
+                Select Operator [SEL_9] (rows=791 width=95)
                   Output:["_col0","_col1"]
-                  Join Operator [JOIN_8] (rows=809 width=178)
+                  Join Operator [JOIN_8] (rows=791 width=178)
                     Output:["_col0","_col2"],condition map:[{"":"{\"type\":\"Inner\",\"left\":0,\"right\":1}"}],keys:{"0":"_col0","1":"_col0"}
                   <-Map 1 [PARTITION-LEVEL SORT]
                     PARTITION-LEVEL SORT [RS_6]
@@ -5241,7 +5241,7 @@ Stage-2
           Map 4
           File Output Operator [FS_11]
             table:{"name:":"default.dest_j1_n14"}
-            Select Operator [SEL_9] (rows=809 width=95)
+            Select Operator [SEL_9] (rows=791 width=95)
               Output:["_col0","_col1"]
               Map Join Operator [MAPJOIN_16]
                 Conds:TS_14.reducesinkkey0=TS_14.reducesinkkey0(Inner),Output:["_col0","_col2"]
@@ -5496,7 +5496,7 @@ Stage-0
                 Stage-1
                   Reducer 2
                   File Output Operator [FS_10]
-                    Join Operator [JOIN_8] (rows=809 width=356)
+                    Join Operator [JOIN_8] (rows=791 width=356)
                       Output:["_col0","_col1","_col2","_col3"],condition map:[{"":"{\"type\":\"Inner\",\"left\":0,\"right\":1}"}],keys:{"0":"_col0","1":"_col0"}
                     <-Map 1 [PARTITION-LEVEL SORT]
                       PARTITION-LEVEL SORT [RS_6]

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/tez/explainanalyze_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_1.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_1.q.out
index 070fca7..c253fd2 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_1.q.out
@@ -251,7 +251,7 @@ Stage-0
     Stage-1
       Reducer 2
       File Output Operator [FS_5]
-        Group By Operator [GBY_3] (rows=309/309 width=95)
+        Group By Operator [GBY_3] (rows=316/309 width=95)
           Output:["_col0","_col1"],aggregations:["count(KEY._col0)"],keys:KEY._col0
         <-Map 1 [SIMPLE_EDGE]
           SHUFFLE [RS_2]
@@ -298,7 +298,7 @@ Stage-0
           Output:["_col0"],aggregations:["count()"]
         <-Reducer 2 [CUSTOM_SIMPLE_EDGE]
           PARTITION_ONLY_SHUFFLE [RS_10]
-            Merge Join Operator [MERGEJOIN_18] (rows=267/0 width=8)
+            Merge Join Operator [MERGEJOIN_18] (rows=262/0 width=8)
               Conds:RS_6._col0=RS_7._col0(Inner)
             <-Map 1 [SIMPLE_EDGE]
               SHUFFLE [RS_6]
@@ -347,7 +347,7 @@ Stage-0
           Output:["_col0"],aggregations:["count()"]
         <-Reducer 2 [CUSTOM_SIMPLE_EDGE]
           PARTITION_ONLY_SHUFFLE [RS_10]
-            Merge Join Operator [MERGEJOIN_18] (rows=267/1019 width=8)
+            Merge Join Operator [MERGEJOIN_18] (rows=262/1019 width=8)
               Conds:RS_6._col0=RS_7._col0(Inner)
             <-Map 1 [SIMPLE_EDGE]
               SHUFFLE [RS_6]
@@ -451,9 +451,9 @@ Stage-0
     Stage-1
       Reducer 2
       File Output Operator [FS_10]
-        Select Operator [SEL_9] (rows=809/1028 width=178)
+        Select Operator [SEL_9] (rows=791/1028 width=178)
           Output:["_col0","_col1"]
-          Merge Join Operator [MERGEJOIN_15] (rows=809/1028 width=178)
+          Merge Join Operator [MERGEJOIN_15] (rows=791/1028 width=178)
             Conds:RS_6._col0=RS_7._col0(Inner),Output:["_col0","_col2"]
           <-Map 1 [SIMPLE_EDGE]
             SHUFFLE [RS_6]
@@ -513,9 +513,9 @@ Stage-0
     Stage-1
       Reducer 2
       File Output Operator [FS_9]
-        Transform Operator [SCR_8] (rows=809/1028 width=178)
+        Transform Operator [SCR_8] (rows=791/1028 width=178)
           command:cat
-          Merge Join Operator [MERGEJOIN_14] (rows=809/1028 width=178)
+          Merge Join Operator [MERGEJOIN_14] (rows=791/1028 width=178)
             Conds:RS_3.key=RS_5.key(Inner),Output:["_col0","_col1"]
           <-Map 1 [SIMPLE_EDGE]
             SHUFFLE [RS_3]

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
index bccfa04..42bad01 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out
@@ -847,9 +847,9 @@ Stage-0
     Stage-1
       Map 2 vectorized
       File Output Operator [FS_34]
-        Select Operator [SEL_33] (rows=391/480 width=186)
+        Select Operator [SEL_33] (rows=399/480 width=186)
           Output:["_col0","_col1","_col2"]
-          Map Join Operator [MAPJOIN_32] (rows=391/480 width=186)
+          Map Join Operator [MAPJOIN_32] (rows=399/480 width=186)
             BucketMapJoin:true,Conds:RS_29._col0=SEL_31._col0(Inner),HybridGraceHashJoin:true,Output:["_col0","_col1","_col3"]
           <-Map 1 [CUSTOM_EDGE] vectorized
             MULTICAST [RS_29]

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
index 5c17512..9d14557 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
@@ -44,11 +44,11 @@ Stage-0
     Stage-1
       Reducer 3
       File Output Operator [FS_12]
-        Select Operator [SEL_11] (rows=2076/10 width=553)
+        Select Operator [SEL_11] (rows=2048/10 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]
           SHUFFLE [RS_10]
-            Merge Join Operator [MERGEJOIN_17] (rows=2076/10 width=553)
+            Merge Join Operator [MERGEJOIN_17] (rows=2048/10 width=552)
               Conds:RS_6._col2=RS_7._col2(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"]
             <-Map 1 [SIMPLE_EDGE]
               SHUFFLE [RS_6]
@@ -143,7 +143,7 @@ Stage-0
           Output:["_col0"],aggregations:["count()"]
         <-Reducer 2 [CUSTOM_SIMPLE_EDGE]
           PARTITION_ONLY_SHUFFLE [RS_10]
-            Merge Join Operator [MERGEJOIN_18] (rows=2076/10 width=8)
+            Merge Join Operator [MERGEJOIN_18] (rows=2048/10 width=8)
               Conds:RS_6._col0=RS_7._col0(Inner)
             <-Map 1 [SIMPLE_EDGE]
               SHUFFLE [RS_6]
@@ -232,16 +232,16 @@ Stage-0
     Stage-1
       Reducer 4
       File Output Operator [FS_15]
-        Select Operator [SEL_14] (rows=623/5 width=11)
+        Select Operator [SEL_14] (rows=631/5 width=11)
           Output:["_col0","_col1"]
         <-Reducer 3 [SIMPLE_EDGE]
           SHUFFLE [RS_13]
-            Group By Operator [GBY_11] (rows=623/5 width=11)
+            Group By Operator [GBY_11] (rows=631/5 width=11)
               Output:["_col0","_col1"],aggregations:["count()"],keys:KEY._col0
             <-Reducer 2 [SIMPLE_EDGE]
               SHUFFLE [RS_10]
                 PartitionCols:_col0
-                Merge Join Operator [MERGEJOIN_20] (rows=2076/10 width=3)
+                Merge Join Operator [MERGEJOIN_20] (rows=2048/10 width=3)
                   Conds:RS_6._col1=RS_7._col0(Inner),Output:["_col0"]
                 <-Map 1 [SIMPLE_EDGE]
                   SHUFFLE [RS_6]

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
index fd71c0c..75f29fa 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
@@ -114,9 +114,9 @@ Stage-3
                 <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
                   File Output Operator [FS_19]
                     table:{"name:":"default.src_multi2_n7"}
-                    Select Operator [SEL_18] (rows=849/508 width=178)
+                    Select Operator [SEL_18] (rows=830/508 width=178)
                       Output:["_col0","_col1"]
-                      Merge Join Operator [MERGEJOIN_26] (rows=849/508 width=178)
+                      Merge Join Operator [MERGEJOIN_26] (rows=830/508 width=178)
                         Conds:RS_15._col0=RS_16._col0(Inner),Output:["_col0","_col3"]
                       <-Map 7 [SIMPLE_EDGE]
                         SHUFFLE [RS_16]
@@ -154,7 +154,7 @@ Stage-3
                                       TableScan [TS_3] (rows=25/25 width=175)
                                         Output:["key","value"]
                   PARTITION_ONLY_SHUFFLE [RS_2]
-                    Select Operator [SEL_1] (rows=849/508 width=178)
+                    Select Operator [SEL_1] (rows=830/508 width=178)
                       Output:["key","value"]
                        Please refer to the previous Select Operator [SEL_18]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
index 8b7b11d..d993905 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
@@ -675,9 +675,9 @@ Stage-0
     Stage-1
       Map 2 vectorized
       File Output Operator [FS_34]
-        Select Operator [SEL_33] (rows=391 width=186)
+        Select Operator [SEL_33] (rows=399 width=186)
           Output:["_col0","_col1","_col2"]
-          Map Join Operator [MAPJOIN_32] (rows=391 width=186)
+          Map Join Operator [MAPJOIN_32] (rows=399 width=186)
             BucketMapJoin:true,Conds:RS_29._col0=SEL_31._col0(Inner),HybridGraceHashJoin:true,Output:["_col0","_col1","_col3"]
           <-Map 1 [CUSTOM_EDGE] vectorized
             MULTICAST [RS_29]

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_1.q.out b/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_1.q.out
index 5b9149c..910a812 100644
--- a/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_1.q.out
@@ -56,7 +56,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 25057 Data size: 200456 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash
@@ -175,7 +175,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 25057 Data size: 200456 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
                         HybridGraceHashJoin: true
                         Group By Operator
                           aggregations: count()
@@ -293,7 +293,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 18702 Data size: 149616 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 18464 Data size: 147712 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash
@@ -408,7 +408,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 18702 Data size: 149616 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 18464 Data size: 147712 Basic stats: COMPLETE Column stats: COMPLETE
                         HybridGraceHashJoin: true
                         Group By Operator
                           aggregations: count()
@@ -521,7 +521,7 @@ STAGE PLANS:
                         1 _col0 (type: int)
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 25057 Data size: 200456 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -630,7 +630,7 @@ STAGE PLANS:
                         1 _col0 (type: int)
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 25057 Data size: 200456 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Group By Operator
                         aggregations: count()

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_2.q.out b/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_2.q.out
index 3bacb4a..a3a77f9 100644
--- a/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_2.q.out
@@ -72,7 +72,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 4
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -200,7 +200,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 4
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Group By Operator
                         aggregations: count()
@@ -343,7 +343,7 @@ STAGE PLANS:
                         0 Map 1
                         2 Map 4
                         3 Map 5
-                      Statistics: Num rows: 1694 Data size: 13552 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1584 Data size: 12672 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -492,7 +492,7 @@ STAGE PLANS:
                         0 Map 1
                         2 Map 4
                         3 Map 5
-                      Statistics: Num rows: 1694 Data size: 13552 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1584 Data size: 12672 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Group By Operator
                         aggregations: count()
@@ -671,7 +671,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 6
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -729,7 +729,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 7
                         2 Map 10
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 265 Data size: 2120 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         mode: hash
@@ -915,7 +915,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 6
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Group By Operator
                         aggregations: count()
@@ -974,7 +974,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 7
                         2 Map 10
-                      Statistics: Num rows: 261 Data size: 2088 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 265 Data size: 2120 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Group By Operator
                         aggregations: count()
@@ -1157,7 +1157,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 4
-                      Statistics: Num rows: 20 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 19 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1169,7 +1169,7 @@ STAGE PLANS:
                         input vertices:
                           1 Map 5
                           2 Map 6
-                        Statistics: Num rows: 204 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 196 Data size: 1568 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           mode: hash
@@ -1347,7 +1347,7 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                         2 Map 4
-                      Statistics: Num rows: 20 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 19 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE
                       HybridGraceHashJoin: true
                       Map Join Operator
                         condition map:
@@ -1360,7 +1360,7 @@ STAGE PLANS:
                         input vertices:
                           1 Map 5
                           2 Map 6
-                        Statistics: Num rows: 204 Data size: 1632 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 196 Data size: 1568 Basic stats: COMPLETE Column stats: COMPLETE
                         HybridGraceHashJoin: true
                         Group By Operator
                           aggregations: count()

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/tez/tez-tag.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/tez-tag.q.out b/ql/src/test/results/clientpositive/tez/tez-tag.q.out
index 55ce485..cf96067 100644
--- a/ql/src/test/results/clientpositive/tez/tez-tag.q.out
+++ b/ql/src/test/results/clientpositive/tez/tez-tag.q.out
@@ -190,7 +190,7 @@ Stage-0
           PARTITION_ONLY_SHUFFLE [RS_17]
             Group By Operator [GBY_16] (rows=1 width=8)
               Output:["_col0"],aggregations:["count()"]
-              Merge Join Operator [MERGEJOIN_30] (rows=63 width=8)
+              Merge Join Operator [MERGEJOIN_30] (rows=64 width=8)
                 Conds:RS_12._col0=RS_13._col0(Inner)
               <-Map 6 [SIMPLE_EDGE]
                 SHUFFLE [RS_13]

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/NumDistinctValueEstimatorFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/NumDistinctValueEstimatorFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/NumDistinctValueEstimatorFactory.java
index 4e4dfb7..b630fa3 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/NumDistinctValueEstimatorFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/NumDistinctValueEstimatorFactory.java
@@ -25,6 +25,7 @@ import java.util.Arrays;
 import org.apache.hadoop.hive.common.ndv.fm.FMSketch;
 import org.apache.hadoop.hive.common.ndv.fm.FMSketchUtils;
 import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
+import org.apache.hadoop.hive.common.ndv.hll.HyperLogLogUtils;
 
 public class NumDistinctValueEstimatorFactory {
 
@@ -44,7 +45,7 @@ public class NumDistinctValueEstimatorFactory {
       if (isFMSketch(buf)) {
         return FMSketchUtils.deserializeFM(buf);
       } else {
-        return HyperLogLog.builder().build().deserialize(buf);
+        return HyperLogLogUtils.deserializeHLL(buf);
       }
     } catch (IOException e) {
       throw new RuntimeException(e);
@@ -56,7 +57,7 @@ public class NumDistinctValueEstimatorFactory {
     if (n instanceof FMSketch) {
       return new FMSketch(((FMSketch) n).getNumBitVectors());
     } else {
-      return HyperLogLog.builder().build();
+      return HyperLogLog.builder().setSizeOptimized().build();
     }
   }
 
@@ -65,7 +66,7 @@ public class NumDistinctValueEstimatorFactory {
     if ("fm".equals(func.toLowerCase())) {
       return new FMSketch(numBitVectors);
     } else if ("hll".equals(func.toLowerCase())) {
-      return HyperLogLog.builder().build();
+      return HyperLogLog.builder().setSizeOptimized().build();
     } else {
       throw new RuntimeException("Can not recognize " + func);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HLLDenseRegister.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HLLDenseRegister.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HLLDenseRegister.java
index 12897fc..422bfbe 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HLLDenseRegister.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HLLDenseRegister.java
@@ -62,6 +62,31 @@ public class HLLDenseRegister implements HLLRegister {
     return set(registerIdx, (byte) lr);
   }
 
+  // this is a lossy invert of the function above, which produces a hashcode
+  // which collides with the current winner of the register (we lose all higher 
+  // bits, but we get all bits useful for lesser p-bit options)
+
+  // +-------------|-------------+
+  // |xxxx100000000|1000000000000|  (lr=9 + idx=1024)
+  // +-------------|-------------+
+  //                \
+  // +---------------|-----------+
+  // |xxxx10000000010|00000000000|  (lr=2 + idx=0)
+  // +---------------|-----------+
+
+  // This shows the relevant bits of the original hash value
+  // and how the conversion is moving bits from the index value
+  // over to the leading zero computation
+
+  public void extractLowBitsTo(HLLRegister dest) {
+    for (int idx = 0; idx < register.length; idx++) {
+      byte lr = register[idx]; // this can be a max of 65, never > 127
+      if (lr != 0) {
+        dest.add((long) ((1 << (p + lr - 1)) | idx));
+      }
+    }
+  }
+
   public boolean set(int idx, byte value) {
     boolean updated = false;
     if (idx < register.length && value > register[idx]) {

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HLLSparseRegister.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HLLSparseRegister.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HLLSparseRegister.java
index d62b858..d5ac54a 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HLLSparseRegister.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HLLSparseRegister.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.common.ndv.hll;
 
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.TreeMap;
 
 public class HLLSparseRegister implements HLLRegister {
@@ -187,6 +188,18 @@ public class HLLSparseRegister implements HLLRegister {
     return sparseMap;
   }
 
+  // this is effectively the same as the dense register impl.
+  public void extractLowBitsTo(HLLRegister dest) {
+    for (Entry<Integer, Byte> entry : getSparseMap().entrySet()) {
+      int idx = entry.getKey();
+      byte lr = entry.getValue(); // this can be a max of 65, never > 127
+      if (lr != 0) {
+        // should be a no-op for sparse
+        dest.add((long) ((1 << (p + lr - 1)) | idx));
+      }
+    }
+  }
+
   public int getP() {
     return p;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HyperLogLog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HyperLogLog.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HyperLogLog.java
index a3cc989..91a6865 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HyperLogLog.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HyperLogLog.java
@@ -18,10 +18,8 @@
 
 package org.apache.hadoop.hive.common.ndv.hll;
 
-import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.util.Map;
@@ -160,6 +158,13 @@ public class HyperLogLog implements NumDistinctValueEstimator {
       return this;
     }
 
+    public HyperLogLogBuilder setSizeOptimized() {
+      // allowing this to be increased via config breaks the merge impl
+      // p=10 = ~1kb per vector or smaller
+      this.numRegisterIndexBits = 10;
+      return this;
+    }
+
     public HyperLogLogBuilder setEncoding(EncodingType enc) {
       this.encoding = enc;
       return this;
@@ -431,12 +436,23 @@ public class HyperLogLog implements NumDistinctValueEstimator {
    * @throws IllegalArgumentException
    */
   public void merge(HyperLogLog hll) {
-    if (p != hll.p || chosenHashBits != hll.chosenHashBits) {
+    if (chosenHashBits != hll.chosenHashBits) {
       throw new IllegalArgumentException(
           "HyperLogLog cannot be merged as either p or hashbits are different. Current: "
               + toString() + " Provided: " + hll.toString());
     }
 
+    if (p > hll.p) {
+      throw new IllegalArgumentException(
+          "HyperLogLog cannot merge a smaller p into a larger one : "
+              + toString() + " Provided: " + hll.toString());
+    }
+
+    if (p != hll.p) {
+      // invariant: p > hll.p
+      hll = hll.squash(p);
+    }
+
     EncodingType otherEncoding = hll.getEncoding();
 
     if (encoding.equals(EncodingType.SPARSE) && otherEncoding.equals(EncodingType.SPARSE)) {
@@ -464,7 +480,37 @@ public class HyperLogLog implements NumDistinctValueEstimator {
   }
 
   /**
-   * Converts sparse to dense hll register
+   * Reduces the accuracy of the HLL provided to a smaller size
+   * @param p0 
+   *         - new p size for the new HyperLogLog (smaller or no change)
+   * @return reduced (or same) HyperLogLog instance
+   */
+  public HyperLogLog squash(final int p0) {
+    if (p0 > p) {
+      throw new IllegalArgumentException(
+          "HyperLogLog cannot be be squashed to be bigger. Current: "
+              + toString() + " Provided: " + p0);
+    }
+
+    if (p0 == p) {
+      return this;
+    }
+
+    final HyperLogLog hll = new HyperLogLogBuilder()
+        .setNumRegisterIndexBits(p0).setEncoding(EncodingType.DENSE)
+        .enableNoBias(noBias).build();
+    final HLLDenseRegister result = hll.denseRegister;
+
+    if (encoding == EncodingType.SPARSE) {
+      sparseRegister.extractLowBitsTo(result);
+    } else if (encoding == EncodingType.DENSE) {
+      denseRegister.extractLowBitsTo(result);
+    }
+    return hll;
+  }
+
+  /**
+   * Converts sparse to dense hll register.
    * @param sparseRegister
    *          - sparse register to be converted
    * @return converted dense register
@@ -576,14 +622,7 @@ public class HyperLogLog implements NumDistinctValueEstimator {
 
   @Override
   public NumDistinctValueEstimator deserialize(byte[] buf) {
-    InputStream is = new ByteArrayInputStream(buf);
-    try {
-      HyperLogLog result = HyperLogLogUtils.deserializeHLL(is);
-      is.close();
-      return result;
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
+    return HyperLogLogUtils.deserializeHLL(buf);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HyperLogLogUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HyperLogLogUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HyperLogLogUtils.java
index 4e6510b..aeba2e9 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HyperLogLogUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/common/ndv/hll/HyperLogLogUtils.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.common.ndv.hll;
 
+import java.io.ByteArrayInputStream;
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -126,7 +127,7 @@ public class HyperLogLogUtils {
   }
 
   /**
-   * Refer serializeHLL() for format of serialization. This funtions
+   * Refer serializeHLL() for format of serialization. This function
    * deserializes the serialized hyperloglogs
    * @param in
    *          - input stream
@@ -198,6 +199,22 @@ public class HyperLogLogUtils {
     return result;
   }
 
+  /**
+   * This function deserializes the serialized hyperloglogs from a byte array.
+   * @param buf - to deserialize
+   * @return HyperLogLog
+   */
+  public static HyperLogLog deserializeHLL(final byte[] buf) {
+    InputStream is = new ByteArrayInputStream(buf); // TODO: use faster non-sync inputstream
+    try {
+      HyperLogLog result = deserializeHLL(is);
+      is.close();
+      return result;
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
   private static void bitpackHLLRegister(OutputStream out, byte[] register, int bitWidth)
       throws IOException {
     int bitsLeft = 8;

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/standalone-metastore/src/test/java/org/apache/hadoop/hive/common/ndv/hll/TestHyperLogLog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/common/ndv/hll/TestHyperLogLog.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/common/ndv/hll/TestHyperLogLog.java
index 617d9c3..e014fb5 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/common/ndv/hll/TestHyperLogLog.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/common/ndv/hll/TestHyperLogLog.java
@@ -37,14 +37,18 @@ public class TestHyperLogLog {
     HyperLogLog hll3 = HyperLogLog.builder().setEncoding(EncodingType.DENSE).build();
     HyperLogLog hll4 = HyperLogLog.builder().setNumRegisterIndexBits(16)
         .setEncoding(EncodingType.DENSE).build();
+    HyperLogLog hll5 = HyperLogLog.builder().setNumRegisterIndexBits(12)
+        .setEncoding(EncodingType.DENSE).build();
     int size = 1000;
     for (int i = 0; i < size; i++) {
       hll.addLong(i);
       hll2.addLong(size + i);
       hll3.addLong(2 * size + i);
+      hll4.addLong(3 * size + i);
     }
     double threshold = size > 40000 ? longRangeTolerance : shortRangeTolerance;
     double delta = threshold * size / 100;
+    double delta4 = threshold * (4*size) / 100;
     assertEquals((double) size, (double) hll.count(), delta);
     assertEquals((double) size, (double) hll2.count(), delta);
 
@@ -63,8 +67,13 @@ public class TestHyperLogLog {
     assertEquals((double) 3 * size, (double) hll.count(), delta);
     assertEquals(EncodingType.DENSE, hll.getEncoding());
 
-    // invalid merge -- register set size doesn't match
+    // valid merge -- register set size gets bigger (also 4k items 
     hll.merge(hll4);
+    assertEquals((double) 4 * size, (double) hll.count(), delta4);
+    assertEquals(EncodingType.DENSE, hll.getEncoding());
+
+    // invalid merge -- smaller register merge to bigger
+    hll.merge(hll5);
   }
 
   @Test(expected = IllegalArgumentException.class)
@@ -74,14 +83,18 @@ public class TestHyperLogLog {
     HyperLogLog hll3 = HyperLogLog.builder().setEncoding(EncodingType.SPARSE).build();
     HyperLogLog hll4 = HyperLogLog.builder().setNumRegisterIndexBits(16)
         .setEncoding(EncodingType.SPARSE).build();
+    HyperLogLog hll5 = HyperLogLog.builder().setNumRegisterIndexBits(12)
+        .setEncoding(EncodingType.SPARSE).build();
     int size = 500;
     for (int i = 0; i < size; i++) {
       hll.addLong(i);
       hll2.addLong(size + i);
       hll3.addLong(2 * size + i);
+      hll4.addLong(3 * size + i);
     }
     double threshold = size > 40000 ? longRangeTolerance : shortRangeTolerance;
     double delta = threshold * size / 100;
+    double delta4 = threshold * (4*size) / 100;
     assertEquals((double) size, (double) hll.count(), delta);
     assertEquals((double) size, (double) hll2.count(), delta);
 
@@ -100,8 +113,13 @@ public class TestHyperLogLog {
     assertEquals((double) 3 * size, (double) hll.count(), delta);
     assertEquals(EncodingType.SPARSE, hll.getEncoding());
 
-    // invalid merge -- register set size doesn't match
+    // valid merge -- register set size gets bigger & dense automatically
     hll.merge(hll4);
+    assertEquals((double) 4 * size, (double) hll.count(), delta4);
+    assertEquals(EncodingType.DENSE, hll.getEncoding());
+
+    // invalid merge -- smaller register merge to bigger
+    hll.merge(hll5);
   }
 
   @Test(expected = IllegalArgumentException.class)
@@ -111,11 +129,14 @@ public class TestHyperLogLog {
     HyperLogLog hll3 = HyperLogLog.builder().setEncoding(EncodingType.DENSE).build();
     HyperLogLog hll4 = HyperLogLog.builder().setNumRegisterIndexBits(16)
         .setEncoding(EncodingType.DENSE).build();
+    HyperLogLog hll5 = HyperLogLog.builder().setNumRegisterIndexBits(12)
+        .setEncoding(EncodingType.DENSE).build();
     int size = 1000;
     for (int i = 0; i < size; i++) {
       hll.addLong(i);
       hll2.addLong(size + i);
       hll3.addLong(2 * size + i);
+      hll4.addLong(3 * size + i);
     }
     double threshold = size > 40000 ? longRangeTolerance : shortRangeTolerance;
     double delta = threshold * size / 100;
@@ -137,8 +158,13 @@ public class TestHyperLogLog {
     assertEquals((double) 3 * size, (double) hll.count(), delta);
     assertEquals(EncodingType.DENSE, hll.getEncoding());
 
-    // invalid merge -- register set size doesn't match
-    hll.merge(hll4);
+    // merge should convert hll2 to DENSE
+    hll2.merge(hll4);
+    assertEquals((double) 2 * size, (double) hll2.count(), delta);
+    assertEquals(EncodingType.DENSE, hll2.getEncoding());
+
+    // invalid merge -- smaller register merge to bigger
+    hll.merge(hll5);
   }
 
   @Test(expected = IllegalArgumentException.class)
@@ -148,11 +174,14 @@ public class TestHyperLogLog {
     HyperLogLog hll3 = HyperLogLog.builder().setEncoding(EncodingType.SPARSE).build();
     HyperLogLog hll4 = HyperLogLog.builder().setNumRegisterIndexBits(16)
         .setEncoding(EncodingType.SPARSE).build();
+    HyperLogLog hll5 = HyperLogLog.builder().setNumRegisterIndexBits(12)
+        .setEncoding(EncodingType.SPARSE).build();
     int size = 1000;
     for (int i = 0; i < size; i++) {
       hll.addLong(i);
       hll2.addLong(size + i);
       hll3.addLong(2 * size + i);
+      hll4.addLong(3 * size + i);
     }
     double threshold = size > 40000 ? longRangeTolerance : shortRangeTolerance;
     double delta = threshold * size / 100;
@@ -174,8 +203,14 @@ public class TestHyperLogLog {
     assertEquals((double) 3 * size, (double) hll.count(), delta);
     assertEquals(EncodingType.DENSE, hll.getEncoding());
 
-    // invalid merge -- register set size doesn't match
-    hll.merge(hll4);
+    // merge should convert hll3 to DENSE
+    hll3.merge(hll4);
+    assertEquals((double) 2 * size, (double) hll3.count(), delta);
+    assertEquals(EncodingType.DENSE, hll3.getEncoding());
+
+    // invalid merge -- smaller register merge to bigger
+    hll.merge(hll5);
+
   }
 
   @Test(expected = IllegalArgumentException.class)
@@ -185,11 +220,14 @@ public class TestHyperLogLog {
     HyperLogLog hll3 = HyperLogLog.builder().setEncoding(EncodingType.SPARSE).build();
     HyperLogLog hll4 = HyperLogLog.builder().setNumRegisterIndexBits(16)
         .setEncoding(EncodingType.SPARSE).build();
+    HyperLogLog hll5 = HyperLogLog.builder().setNumRegisterIndexBits(12)
+        .setEncoding(EncodingType.SPARSE).build();
     int size = 1000;
     for (int i = 0; i < size; i++) {
       hll.addLong(i);
       hll2.addLong(size + i);
       hll3.addLong(2 * size + i);
+      hll4.addLong(3 * size + i);
     }
     double threshold = size > 40000 ? longRangeTolerance : shortRangeTolerance;
     double delta = threshold * size / 100;
@@ -211,8 +249,13 @@ public class TestHyperLogLog {
     assertEquals((double) 3 * size, (double) hll.count(), delta);
     assertEquals(EncodingType.DENSE, hll.getEncoding());
 
-    // invalid merge -- register set size doesn't match
-    hll.merge(hll4);
+    // merge should convert hll2 to DENSE
+    hll2.merge(hll4);
+    assertEquals((double) 2 * size, (double) hll2.count(), delta);
+    assertEquals(EncodingType.DENSE, hll2.getEncoding());
+
+    // invalid merge -- smaller register merge to bigger
+    hll.merge(hll5);
   }
 
   @Test
@@ -227,4 +270,69 @@ public class TestHyperLogLog {
     double delta = threshold * size / 100;
     assertEquals((double) size, (double) hll.count(), delta);
   }
+
+  @Test
+  public void testHLLSquash() {
+
+    int[] sizes = new int[] { 500, 1000, 2300, 4096};
+    int minBits = 9;
+    for (final int size : sizes) {
+
+      HyperLogLog hlls[] = new HyperLogLog[16];
+      for (int k = minBits; k < hlls.length; k++) {
+        final HyperLogLog hll = HyperLogLog.builder()
+            .setEncoding(EncodingType.DENSE).setNumRegisterIndexBits(k).build();
+        for (int i = 0; i < size; i++) {
+          hll.addLong(i);
+        }
+        hlls[k] = hll;
+      }
+
+      for (int k = minBits; k < hlls.length; k++) {
+        for (int j = k + 1; j < hlls.length; j++) {
+          final HyperLogLog large = hlls[j];
+          final HyperLogLog small = hlls[k];
+          final HyperLogLog mush = large
+              .squash(small.getNumRegisterIndexBits());
+          assertEquals(small.count(), mush.count(), 0);
+          double delta = Math.ceil(small.getStandardError()*size);
+          assertEquals((double) size, (double) mush.count(), delta);
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testHLLDenseDenseSquash() {
+    HyperLogLog p14HLL = HyperLogLog.builder().setEncoding(EncodingType.DENSE).setNumRegisterIndexBits(14).build();
+    HyperLogLog p10HLL = HyperLogLog.builder().setEncoding(EncodingType.DENSE).setNumRegisterIndexBits(10).build();
+    int size = 1_000_000;
+    for (int i = 0; i < size; i++) {
+      p14HLL.addLong(i);
+    }
+
+    for (int i = 0; i < 10_000; i++) {
+      p10HLL.addLong(i);
+    }
+
+    p14HLL.squash(p10HLL.getNumRegisterIndexBits());
+    assertEquals((double) size, p14HLL.count(), longRangeTolerance * size / 100.0);
+  }
+
+  @Test
+  public void testHLLSparseDenseSquash() {
+    HyperLogLog p14HLL = HyperLogLog.builder().setEncoding(EncodingType.SPARSE).setNumRegisterIndexBits(14).build();
+    HyperLogLog p10HLL = HyperLogLog.builder().setEncoding(EncodingType.DENSE).setNumRegisterIndexBits(10).build();
+    int size = 2000;
+    for (int i = 0; i < size; i++) {
+      p14HLL.addLong(i);
+    }
+
+    for (int i = 0; i < 10_000; i++) {
+      p10HLL.addLong(i);
+    }
+
+    p14HLL.squash(p10HLL.getNumRegisterIndexBits());
+    assertEquals((double) size, p14HLL.count(), longRangeTolerance * size / 100.0);
+  }
 }


[05/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/sharedworkext.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/sharedworkext.q.out b/ql/src/test/results/clientpositive/llap/sharedworkext.q.out
index 1837019..c734d00 100644
--- a/ql/src/test/results/clientpositive/llap/sharedworkext.q.out
+++ b/ql/src/test/results/clientpositive/llap/sharedworkext.q.out
@@ -88,22 +88,22 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   keys: _col2 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 70310 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: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 70310 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: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -111,16 +111,16 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 70310 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: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -131,10 +131,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 528 Data size: 45936 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 493 Data size: 42891 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 528 Data size: 45936 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 493 Data size: 42891 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -146,16 +146,16 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 70310 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: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -595,17 +595,17 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   keys: _col1 (type: string), _col2 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 70310 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: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -613,28 +613,28 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string)
                   outputColumnNames: _col1
-                  Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: 0 (type: int), _col1 (type: string)
                     sort order: ++
                     Map-reduce partition columns: 0 (type: int)
-                    Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: 0 (type: int), _col1 (type: string)
                     sort order: ++
                     Map-reduce partition columns: 0 (type: int)
-                    Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey1 (type: string)
                 outputColumnNames: _col1
-                Statistics: Num rows: 404 Data size: 35148 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 34365 Basic stats: COMPLETE Column stats: COMPLETE
                 PTF Operator
                   Function definitions:
                       Input definition
@@ -655,19 +655,19 @@ STAGE PLANS:
                               window function: GenericUDAFRankEvaluator
                               window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
                               isPivotResult: true
-                  Statistics: Num rows: 404 Data size: 35148 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 34365 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: rank_window_0 is not null (type: boolean)
-                    Statistics: Num rows: 404 Data size: 35148 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 34365 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: rank_window_0 (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 404 Data size: 1616 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 395 Data size: 1580 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: 404 Data size: 1616 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 395 Data size: 1580 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
@@ -678,10 +678,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 404 Data size: 1616 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 1580 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 404 Data size: 1616 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 1580 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:
               Select Operator
                 expressions: KEY.reducesinkkey1 (type: string)
                 outputColumnNames: _col1
-                Statistics: Num rows: 404 Data size: 35148 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 34365 Basic stats: COMPLETE Column stats: COMPLETE
                 PTF Operator
                   Function definitions:
                       Input definition
@@ -713,19 +713,19 @@ STAGE PLANS:
                               window function: GenericUDAFRankEvaluator
                               window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
                               isPivotResult: true
-                  Statistics: Num rows: 404 Data size: 35148 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 34365 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: rank_window_0 is not null (type: boolean)
-                    Statistics: Num rows: 404 Data size: 35148 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 34365 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: rank_window_0 (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 404 Data size: 1616 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 395 Data size: 1580 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: 404 Data size: 1616 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 395 Data size: 1580 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/skewjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/skewjoin.q.out b/ql/src/test/results/clientpositive/llap/skewjoin.q.out
index a4f6d8f..1c892a3 100644
--- a/ql/src/test/results/clientpositive/llap/skewjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/skewjoin.q.out
@@ -142,14 +142,14 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col2
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: UDFToInteger(_col0) (type: int), _col2 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -158,7 +158,7 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: int), _col1 (type: string)
                     outputColumnNames: key, value
-                    Statistics: Num rows: 809 Data size: 76855 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                       mode: hash
@@ -688,7 +688,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col2, _col3
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(hash(_col2)), sum(hash(_col3))
                   mode: hash
@@ -815,7 +815,7 @@ STAGE PLANS:
                   0 _col0 (type: string), UDFToDouble(substring(_col1, 5)) (type: double)
                   1 _col0 (type: string), (substring(_col1, 5) + 1) (type: double)
                 outputColumnNames: _col2, _col3
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(hash(_col2)), sum(hash(_col3))
                   mode: hash
@@ -971,7 +971,7 @@ STAGE PLANS:
                   1 _col0 (type: string)
                   2 _col0 (type: string)
                 outputColumnNames: _col0, _col3
-                Statistics: Num rows: 143 Data size: 25454 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 135 Data size: 24030 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: sum(hash(_col0)), sum(hash(_col3))
                   mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/smb_mapjoin_6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/smb_mapjoin_6.q.out b/ql/src/test/results/clientpositive/llap/smb_mapjoin_6.q.out
index f44a0db..3e5acd0 100644
--- a/ql/src/test/results/clientpositive/llap/smb_mapjoin_6.q.out
+++ b/ql/src/test/results/clientpositive/llap/smb_mapjoin_6.q.out
@@ -127,10 +127,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 153710 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 825 Data size: 156750 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 153710 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 825 Data size: 156750 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -139,7 +139,7 @@ STAGE PLANS:
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string)
                   outputColumnNames: k1, v1, k2, v2
-                  Statistics: Num rows: 809 Data size: 153710 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 825 Data size: 156750 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: compute_stats(k1, 'hll'), compute_stats(v1, 'hll'), compute_stats(k2, 'hll'), compute_stats(v2, 'hll')
                     mode: hash
@@ -1343,10 +1343,10 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 809 Data size: 153710 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 825 Data size: 156750 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 153710 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 825 Data size: 156750 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1355,7 +1355,7 @@ STAGE PLANS:
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string)
                   outputColumnNames: k1, v1, k2, v2
-                  Statistics: Num rows: 809 Data size: 153710 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 825 Data size: 156750 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: compute_stats(k1, 'hll'), compute_stats(v1, 'hll'), compute_stats(k2, 'hll'), compute_stats(v2, 'hll')
                     mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/subquery_corr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/subquery_corr.q.out b/ql/src/test/results/clientpositive/llap/subquery_corr.q.out
index 740df7f..466de2a 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_corr.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_corr.q.out
@@ -70,14 +70,14 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3
                 residual filter predicates: {(_col1 > _col3)}
-                Statistics: Num rows: 134 Data size: 36046 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 35239 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 134 Data size: 23852 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 134 Data size: 23852 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -169,14 +169,14 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3
                 residual filter predicates: {(_col1 <= _col3)}
-                Statistics: Num rows: 134 Data size: 36046 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 35239 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 134 Data size: 23852 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 134 Data size: 23852 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/subquery_exists.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/subquery_exists.q.out b/ql/src/test/results/clientpositive/llap/subquery_exists.q.out
index 7b5def5..21d3d53 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_exists.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_exists.q.out
@@ -81,10 +81,10 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -289,10 +289,10 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 407 Data size: 72446 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 407 Data size: 72446 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/subquery_in.q.out
----------------------------------------------------------------------
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 7dd64af..f961402 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
@@ -74,10 +74,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -195,10 +195,10 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -688,10 +688,10 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -816,14 +816,14 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3
                 residual filter predicates: {(_col1 <> _col3)}
-                Statistics: Num rows: 66 Data size: 17490 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 64 Data size: 16960 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 66 Data size: 11748 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 64 Data size: 11392 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 66 Data size: 11748 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 64 Data size: 11392 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -950,14 +950,14 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3
                 residual filter predicates: {(_col3 > _col1)}
-                Statistics: Num rows: 67 Data size: 18023 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 65 Data size: 17485 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 67 Data size: 11926 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 65 Data size: 11570 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 67 Data size: 11926 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 65 Data size: 11570 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2727,18 +2727,18 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 404 Data size: 35148 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 407 Data size: 35409 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 202 Data size: 19190 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 203 Data size: 19285 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: 202 Data size: 19190 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 203 Data size: 19285 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -2748,15 +2748,15 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 202 Data size: 19190 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 203 Data size: 19285 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: _col1 is not null (type: boolean)
-                  Statistics: Num rows: 202 Data size: 19190 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 203 Data size: 19285 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col1 (type: bigint)
                     sort order: +
                     Map-reduce partition columns: _col1 (type: bigint)
-                    Statistics: Num rows: 202 Data size: 19190 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 203 Data size: 19285 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: string)
         Reducer 4 
             Execution mode: llap
@@ -2948,18 +2948,18 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 404 Data size: 35148 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 34365 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 202 Data size: 19190 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 197 Data size: 18715 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: 202 Data size: 19190 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 197 Data size: 18715 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -2969,15 +2969,15 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 202 Data size: 19190 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 197 Data size: 18715 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: _col1 is not null (type: boolean)
-                  Statistics: Num rows: 202 Data size: 19190 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 197 Data size: 18715 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col1 (type: bigint)
                     sort order: +
                     Map-reduce partition columns: _col1 (type: bigint)
-                    Statistics: Num rows: 202 Data size: 19190 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 197 Data size: 18715 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: string)
         Reducer 4 
             Execution mode: llap
@@ -3479,10 +3479,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 40 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
index 755801c..b4ce6f8 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
@@ -628,18 +628,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 65 Data size: 12090 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: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 65 Data size: 12090 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -649,15 +649,15 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 65 Data size: 12090 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: _col2 is not null (type: boolean)
-                  Statistics: Num rows: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 65 Data size: 12090 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col2 (type: bigint)
                     sort order: +
                     Map-reduce partition columns: _col2 (type: bigint)
-                    Statistics: Num rows: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 65 Data size: 12090 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: string), _col1 (type: string)
         Reducer 4 
             Execution mode: llap
@@ -793,18 +793,18 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           keys: _col0 (type: string), _col1 (type: string)
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 65 Data size: 12090 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: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 65 Data size: 12090 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col2 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -855,10 +855,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 65 Data size: 12090 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: _col2 is not null (type: boolean)
-                  Statistics: Num rows: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 65 Data size: 12090 Basic stats: COMPLETE Column stats: COMPLETE
                   Map Join Operator
                     condition map:
                          Left Semi Join 0 to 1
@@ -962,18 +962,18 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           keys: _col0 (type: string), _col1 (type: string)
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 202 Data size: 37572 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 197 Data size: 36642 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: 202 Data size: 37572 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col2 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -1031,10 +1031,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 202 Data size: 37572 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: _col2 is not null (type: boolean)
-                  Statistics: Num rows: 202 Data size: 37572 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
                   Map Join Operator
                     condition map:
                          Left Semi Join 0 to 1

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
index 8af9d8e..5252684 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
@@ -3224,18 +3224,18 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 404 Data size: 71912 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 202 Data size: 37572 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 197 Data size: 36642 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: 202 Data size: 37572 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -3245,15 +3245,15 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 202 Data size: 37572 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: _col2 is not null (type: boolean)
-                  Statistics: Num rows: 202 Data size: 37572 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col2 (type: bigint)
                     sort order: +
                     Map-reduce partition columns: _col2 (type: bigint)
-                    Statistics: Num rows: 202 Data size: 37572 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: string), _col1 (type: string)
         Reducer 4 
             Execution mode: llap
@@ -3283,7 +3283,7 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 134 Data size: 11658 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 135 Data size: 11745 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
@@ -4117,10 +4117,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2, _col4
-                Statistics: Num rows: 500 Data size: 50040 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 50028 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 500 Data size: 50040 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 50028 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string), _col2 (type: bigint), _col4 (type: boolean)
         Reducer 4 
             Execution mode: llap
@@ -4132,12 +4132,12 @@ STAGE PLANS:
                   0 
                   1 
                 outputColumnNames: _col1, _col2, _col4, _col5
-                Statistics: Num rows: 500 Data size: 52040 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 52028 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: ((_col4 is not null and (_col2 <> 0L)) or _col1 is not null or _col5 is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 52040 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 52028 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 52040 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 52028 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count()
                       mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/subquery_notin.q.out b/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
index 5eabc7d..21a0f84 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
@@ -101,10 +101,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col5
-                Statistics: Num rows: 500 Data size: 97540 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 97528 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 97540 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 97528 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: _col0, _col1
@@ -1512,10 +1512,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col4
-                Statistics: Num rows: 166 Data size: 17366 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 166 Data size: 17358 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: ((_col1 = 0L) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean)
-                  Statistics: Num rows: 166 Data size: 17366 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 166 Data size: 17358 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string)
                     outputColumnNames: _col0
@@ -3581,14 +3581,14 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col5
-                Statistics: Num rows: 500 Data size: 98620 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 98584 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: ((_col2 = 0L) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 98620 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 98584 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string)
                     outputColumnNames: _col0
-                    Statistics: Num rows: 500 Data size: 98620 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 98584 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count()
                       keys: _col0 (type: string)
@@ -3853,12 +3853,12 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3, _col4
-                Statistics: Num rows: 500 Data size: 95480 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 95336 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: 500 Data size: 95480 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 95336 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col3 (type: bigint), _col4 (type: bigint)
         Reducer 3 
             Execution mode: llap
@@ -3870,14 +3870,14 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col1 (type: string), _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3, _col4, _col7
-                Statistics: Num rows: 500 Data size: 97100 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 96920 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: CASE WHEN ((_col3 = 0L)) THEN (true) WHEN (_col3 is null) THEN (true) WHEN (_col7 is not null) THEN (false) WHEN (_col1 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (false) ELSE (true) END (type: boolean)
-                  Statistics: Num rows: 250 Data size: 48560 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 48460 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string)
                     outputColumnNames: _col0
-                    Statistics: Num rows: 250 Data size: 48560 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 250 Data size: 48460 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count()
                       keys: _col0 (type: string)
@@ -4609,10 +4609,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col5
-                Statistics: Num rows: 500 Data size: 97164 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 97160 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 97164 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 97160 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: _col0, _col1
@@ -7054,12 +7054,12 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3, _col4
-                Statistics: Num rows: 500 Data size: 91672 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 91688 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: 500 Data size: 91672 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 91688 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string), _col3 (type: bigint), _col4 (type: bigint)
         Reducer 3 
             Execution mode: llap
@@ -7072,10 +7072,10 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3, _col4, _col6, _col7
                 residual filter predicates: {(_col1 > _col6)}
-                Statistics: Num rows: 500 Data size: 104497 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 104228 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: CASE WHEN ((_col3 = 0L)) THEN (true) WHEN (_col3 is null) THEN (true) WHEN (_col7 is not null) THEN (false) WHEN (_col0 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (false) ELSE (true) END (type: boolean)
-                  Statistics: Num rows: 250 Data size: 52304 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 250 Data size: 52114 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: _col0, _col1

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
index 5e24ae6..ed3c96f 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
@@ -5472,18 +5472,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 133 Data size: 23674 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 65 Data size: 12090 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: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 65 Data size: 12090 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -5493,10 +5493,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 65 Data size: 12090 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 66 Data size: 12276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 65 Data size: 12090 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint)
         Reducer 4 
             Execution mode: llap
@@ -5509,14 +5509,14 @@ STAGE PLANS:
                   1 
                 outputColumnNames: _col0, _col1, _col2, _col3
                 residual filter predicates: {(_col2 > _col3)}
-                Statistics: Num rows: 22 Data size: 4268 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 21 Data size: 4074 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 22 Data size: 4092 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 21 Data size: 3906 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 22 Data size: 4092 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 21 Data size: 3906 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/subquery_select.q.out
----------------------------------------------------------------------
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 f6bcb9f..f21fd39 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_select.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_select.q.out
@@ -1861,10 +1861,10 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 134 Data size: 23852 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 134 Data size: 23852 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2027,10 +2027,10 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 108 Data size: 19224 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 106 Data size: 18868 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 108 Data size: 19224 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 106 Data size: 18868 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2045,11 +2045,11 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 134 Data size: 36850 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 135 Data size: 37125 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col2 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 134 Data size: 36850 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 135 Data size: 37125 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     keys: _col0 (type: string), _col1 (type: string)
                     mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/subquery_views.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/subquery_views.q.out b/ql/src/test/results/clientpositive/llap/subquery_views.q.out
index 4609668..cde0413 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_views.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_views.q.out
@@ -235,19 +235,19 @@ STAGE PLANS:
                   1 _col0 (type: string), _col1 (type: string)
                   2 _col3 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col4, _col5, _col8
-                Statistics: Num rows: 26 Data size: 5148 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 25 Data size: 4950 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: CASE WHEN ((_col4 = 0L)) THEN (true) WHEN (_col4 is null) THEN (true) WHEN (_col8 is not null) THEN (false) WHEN (_col0 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (false) ELSE (true) END (type: boolean)
-                  Statistics: Num rows: 13 Data size: 2574 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12 Data size: 2376 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12 Data size: 2136 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: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 12 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -259,10 +259,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -335,14 +335,14 @@ STAGE PLANS:
                   1 _col0 (type: string), _col1 (type: string)
                   2 _col3 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col4, _col5, _col8
-                Statistics: Num rows: 26 Data size: 2782 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 25 Data size: 2675 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: CASE WHEN ((_col4 = 0L)) THEN (true) WHEN (_col4 is null) THEN (true) WHEN (_col8 is not null) THEN (false) WHEN (_col0 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (false) ELSE (true) END (type: boolean)
-                  Statistics: Num rows: 13 Data size: 1391 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string)
                     outputColumnNames: _col0
-                    Statistics: Num rows: 13 Data size: 1131 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 12 Data size: 1044 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       keys: _col0 (type: string)
                       mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out b/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
index 405b293..5107fbe 100644
--- a/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
@@ -103,11 +103,11 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 1618 Data size: 153710 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1650 Data size: 156750 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
-                          Statistics: Num rows: 1618 Data size: 153710 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1650 Data size: 156750 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: some inputs
@@ -136,10 +136,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1618 Data size: 153710 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1650 Data size: 156750 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1618 Data size: 153710 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1650 Data size: 156750 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_2.q.out
----------------------------------------------------------------------
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 4ccfe1a..3373fc25 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
@@ -105,11 +105,11 @@ STAGE PLANS:
                   1 UDFToInteger(_col0) (type: int)
                   2 (UDFToInteger(_col0) + 0) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 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 3 
             Execution mode: vectorized, llap
@@ -117,10 +117,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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -296,11 +296,11 @@ STAGE PLANS:
                   1 UDFToInteger(_col0) (type: int)
                   2 (UDFToInteger(_col0) + 0) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 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 3 
             Execution mode: vectorized, llap
@@ -308,10 +308,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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -487,11 +487,11 @@ STAGE PLANS:
                   1 UDFToInteger(_col0) (type: int)
                   2 (UDFToInteger(_col0) + 0) (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 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 3 
             Execution mode: vectorized, llap
@@ -499,10 +499,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: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1777 Data size: 312360 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1715 Data size: 293140 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out b/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out
index cce289d..122eb5c 100644
--- a/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out
@@ -82,7 +82,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
-                Statistics: Num rows: 809 Data size: 6472 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 6328 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -207,7 +207,7 @@ STAGE PLANS:
                         key expressions: _col1 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col1 (type: string)
-                        Statistics: Num rows: 14944 Data size: 2660032 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 14658 Data size: 2609124 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -240,16 +240,16 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 12944 Data size: 2304032 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 12658 Data size: 2253124 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col2 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 12944 Data size: 2304032 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 12658 Data size: 2253124 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col1 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col1 (type: string)
-                    Statistics: Num rows: 14944 Data size: 2660032 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 14658 Data size: 2609124 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: string)
         Reducer 4 
             Execution mode: llap
@@ -261,18 +261,18 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 24181 Data size: 4304218 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 23872 Data size: 4249216 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 12090 Data size: 2248740 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11936 Data size: 2220096 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: 12090 Data size: 2248740 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 11936 Data size: 2220096 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
         Reducer 5 
             Execution mode: vectorized, llap
@@ -282,14 +282,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 12090 Data size: 2248740 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 11936 Data size: 2220096 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col2 (type: bigint)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 12090 Data size: 1148550 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 11936 Data size: 1133920 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 12090 Data size: 1148550 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 11936 Data size: 1133920 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat


[11/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/correlationoptimizer1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer1.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer1.q.out
index e91b699..99f3c56 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer1.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer1.q.out
@@ -72,7 +72,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
@@ -220,7 +220,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
@@ -360,7 +360,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           0 Map 1
-                        Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           keys: _col0 (type: string)
@@ -515,18 +515,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 20 Data size: 1720 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 19 Data size: 1634 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 10 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9 Data size: 846 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: 10 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 846 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -536,11 +536,11 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 10 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9 Data size: 846 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: hash(_col0) (type: int), hash(_col1) (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 10 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9 Data size: 846 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1)
                     mode: hash
@@ -668,18 +668,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 20 Data size: 1720 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 19 Data size: 1634 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 10 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9 Data size: 846 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: 10 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 846 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -689,11 +689,11 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 10 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9 Data size: 846 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: hash(_col0) (type: int), hash(_col1) (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 10 Data size: 940 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9 Data size: 846 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1)
                     mode: hash
@@ -810,7 +810,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
@@ -952,7 +952,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
@@ -1094,7 +1094,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1
-                Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col1 (type: string)
@@ -1236,7 +1236,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1
-                Statistics: Num rows: 40 Data size: 3480 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3393 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col1 (type: string)
@@ -1375,18 +1375,18 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col3
-                Statistics: Num rows: 40 Data size: 7080 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6903 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col3 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 20 Data size: 3700 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3515 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: 20 Data size: 3700 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 19 Data size: 3515 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -1396,10 +1396,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 20 Data size: 3700 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 19 Data size: 3515 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 20 Data size: 3700 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3515 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1506,18 +1506,18 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col3
-                Statistics: Num rows: 40 Data size: 7080 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6903 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col3 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 20 Data size: 3700 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3515 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: 20 Data size: 3700 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 19 Data size: 3515 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -1527,10 +1527,10 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 20 Data size: 3700 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 19 Data size: 3515 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 20 Data size: 3700 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3515 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1924,7 +1924,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 500 Data size: 3526 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
@@ -2066,7 +2066,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 500 Data size: 3526 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 500 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
@@ -2208,7 +2208,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 525 Data size: 3526 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 525 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
@@ -2350,7 +2350,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 525 Data size: 3526 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 525 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
@@ -2499,18 +2499,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 20 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3477 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: 20 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 19 Data size: 3477 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -2520,11 +2520,11 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 20 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 19 Data size: 3477 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 20 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3477 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1), sum(_col2)
                     mode: hash
@@ -2648,18 +2648,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 20 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3477 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: 20 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 19 Data size: 3477 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -2669,11 +2669,11 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 20 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 19 Data size: 3477 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 20 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3477 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1), sum(_col2)
                     mode: hash
@@ -2796,7 +2796,7 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
@@ -2944,7 +2944,7 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/correlationoptimizer2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer2.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer2.q.out
index 0f839ea..8e17d95 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer2.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer2.q.out
@@ -1804,16 +1804,16 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string), _col2 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6825 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: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -1825,11 +1825,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 30 Data size: 8070 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 29 Data size: 7801 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int), hash(_col3) (type: int)
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 30 Data size: 8070 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 29 Data size: 7801 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1), sum(_col2), sum(_col3)
                     mode: hash
@@ -1985,16 +1985,16 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string), _col2 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6825 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: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -2006,11 +2006,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 30 Data size: 8070 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 29 Data size: 7801 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int), hash(_col3) (type: int)
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 30 Data size: 8070 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 29 Data size: 7801 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1), sum(_col2), sum(_col3)
                     mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/correlationoptimizer3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer3.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer3.q.out
index 852f391..2e4010e 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer3.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer3.q.out
@@ -101,16 +101,16 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string), _col2 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6825 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: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -122,11 +122,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: hash(_col0) (type: int), hash(_col3) (type: int), hash(_col1) (type: int)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1), sum(_col2)
                     mode: hash
@@ -161,7 +161,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1
-                Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col1 (type: string)
@@ -318,16 +318,16 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string), _col2 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6825 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: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -339,11 +339,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: hash(_col0) (type: int), hash(_col3) (type: int), hash(_col1) (type: int)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1), sum(_col2)
                     mode: hash
@@ -378,7 +378,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1
-                Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col1 (type: string)
@@ -482,11 +482,11 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: string), _col2 (type: string)
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -496,11 +496,11 @@ STAGE PLANS:
                             outputColumnNames: _col0, _col1, _col3
                             input vertices:
                               1 Reducer 5
-                            Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                             Select Operator
                               expressions: hash(_col0) (type: int), hash(_col3) (type: int), hash(_col1) (type: int)
                               outputColumnNames: _col0, _col1, _col2
-                              Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                               Group By Operator
                                 aggregations: sum(_col0), sum(_col1), sum(_col2)
                                 mode: hash
@@ -553,7 +553,7 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 6
-                        Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           keys: _col1 (type: string)
@@ -746,16 +746,16 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string), _col2 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6825 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: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -767,11 +767,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2, _col3
-                Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: hash(_col2) (type: int), hash(_col3) (type: int), hash(_col1) (type: int)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1), sum(_col2)
                     mode: hash
@@ -806,7 +806,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1
-                Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col1 (type: string)
@@ -963,16 +963,16 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string), _col2 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6825 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: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -984,11 +984,11 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1, _col2, _col3
-                Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: hash(_col2) (type: int), hash(_col3) (type: int), hash(_col1) (type: int)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1), sum(_col2)
                     mode: hash
@@ -1023,7 +1023,7 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col1
-                Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col1 (type: string)
@@ -1127,11 +1127,11 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col1 (type: string), _col2 (type: string)
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 40 Data size: 7000 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 39 Data size: 6825 Basic stats: COMPLETE Column stats: COMPLETE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -1141,11 +1141,11 @@ STAGE PLANS:
                             outputColumnNames: _col1, _col2, _col3
                             input vertices:
                               1 Reducer 5
-                            Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                             Select Operator
                               expressions: hash(_col2) (type: int), hash(_col3) (type: int), hash(_col1) (type: int)
                               outputColumnNames: _col0, _col1, _col2
-                              Statistics: Num rows: 40 Data size: 7320 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 39 Data size: 7137 Basic stats: COMPLETE Column stats: COMPLETE
                               Group By Operator
                                 aggregations: sum(_col0), sum(_col1), sum(_col2)
                                 mode: hash
@@ -1198,7 +1198,7 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 6
-                        Statistics: Num rows: 40 Data size: 3440 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           keys: _col1 (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/correlationoptimizer6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer6.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer6.q.out
index 499ef4b..9e424c2 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer6.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer6.q.out
@@ -172,18 +172,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 8 
             Execution mode: vectorized, llap
@@ -193,12 +193,12 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
 
   Stage: Stage-0
@@ -416,18 +416,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 8 
             Execution mode: vectorized, llap
@@ -437,12 +437,12 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
 
   Stage: Stage-0
@@ -593,18 +593,18 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 6
-                        Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           keys: _col0 (type: string)
                           mode: hash
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -650,7 +650,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                 Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -788,10 +788,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 20 Data size: 3620 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 19 Data size: 3439 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 20 Data size: 3620 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3439 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -929,10 +929,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 20 Data size: 3620 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 19 Data size: 3439 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 20 Data size: 3620 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 19 Data size: 3439 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1105,18 +1105,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 5 
             Execution mode: vectorized, llap
@@ -1126,12 +1126,12 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
 
   Stage: Stage-0
@@ -1286,18 +1286,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 5 
             Execution mode: vectorized, llap
@@ -1307,12 +1307,12 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
 
   Stage: Stage-0
@@ -1864,14 +1864,14 @@ STAGE PLANS:
                   1 _col0 (type: string)
                   2 _col0 (type: string)
                 outputColumnNames: _col1, _col2, _col3
-                Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string), _col2 (type: string), _col3 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1886,18 +1886,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 5 
             Execution mode: vectorized, llap
@@ -1907,12 +1907,12 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
 
   Stage: Stage-0
@@ -2082,14 +2082,14 @@ STAGE PLANS:
                   1 _col0 (type: string)
                   2 _col0 (type: string)
                 outputColumnNames: _col1, _col2, _col3
-                Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: string), _col2 (type: string), _col3 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2104,18 +2104,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 5 
             Execution mode: vectorized, llap
@@ -2125,12 +2125,12 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
 
   Stage: Stage-0
@@ -2298,18 +2298,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -2319,12 +2319,12 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
         Reducer 4 
             Execution mode: llap
@@ -2338,14 +2338,14 @@ STAGE PLANS:
                   1 _col0 (type: string)
                   2 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col3 (type: string), _col0 (type: string), _col1 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2516,18 +2516,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
@@ -2537,12 +2537,12 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 316 Data size: 30020 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: 309 Data size: 29355 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
         Reducer 4 
             Execution mode: llap
@@ -2556,14 +2556,14 @@ STAGE PLANS:
                   1 _col0 (type: string)
                   2 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col3 (type: string), _col0 (type: string), _col1 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 40 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 39 Data size: 7059 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3327,18 +3327,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 73470 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: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 73470 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
         Reducer 9 
             Execution mode: vectorized, llap
@@ -3348,12 +3348,12 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 73470 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: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 73470 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string), _col2 (type: bigint)
 
   Stage: Stage-0
@@ -3572,18 +3572,18 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 73470 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: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 395 Data size: 73470 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col2 (type: bigint)
         Reducer 8 
             Execution mode: vectorized, llap
@@ -3593,12 +3593,12 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 73470 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: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 395 Data size: 73470 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string), _col2 (type: bigint)
 
   Stage: Stage-0
@@ -3749,18 +3749,18 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         input vertices:
                           1 Map 6
-                        Statistics: Num rows: 809 Data size: 144002 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
                           keys: _col0 (type: string), _col1 (type: string)
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 395 Data size: 73470 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: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 395 Data size: 73470 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col2 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -3806,7 +3806,7 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 404 Data size: 75144 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 395 Data size: 73470 Basic stats: COMPLETE Column stats: COMPLETE
                 Map Join Operator
                   condition map:
                        Inner Join 0 to 1

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/cross_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/cross_join.q.out b/ql/src/test/results/clientpositive/llap/cross_join.q.out
index e95407d..64f669b 100644
--- a/ql/src/test/results/clientpositive/llap/cross_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/cross_join.q.out
@@ -200,10 +200,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -390,10 +390,10 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 809 Data size: 70383 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out
index 2c448df..8f8665a 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out
@@ -452,12 +452,12 @@ STAGE PLANS:
                           keys: _col0 (type: string)
                           mode: hash
                           outputColumnNames: _col0
-                          Statistics: Num rows: 309 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 316 Data size: 27492 Basic stats: COMPLETE Column stats: COMPLETE
                           Dynamic Partitioning Event Operator
                             Target column: ds (string)
                             Target Input: srcpart_small_n3
                             Partition key expr: ds
-                            Statistics: Num rows: 309 Data size: 26883 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 316 Data size: 27492 Basic stats: COMPLETE Column stats: COMPLETE
                             Target Vertex: Map 4
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -723,7 +723,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 2000 Data size: 174000 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=309)
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=316)
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2
                           Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
@@ -826,7 +826,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=309)
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=316)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw.q.out
index 99163da..6056865 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw.q.out
@@ -231,7 +231,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1000 Data size: 87000 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=309)
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=316)
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2
                           Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
@@ -284,7 +284,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1000 Data size: 87000 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=309)
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=316)
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2
                           Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
@@ -369,7 +369,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=309)
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=316)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
@@ -430,7 +430,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=309)
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=316)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/dynamic_semijoin_user_level.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_user_level.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_user_level.q.out
index eefa592..c3ef505 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_user_level.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_user_level.q.out
@@ -256,8 +256,8 @@ Stage-0
                       predicate:key is not null
                       TableScan [TS_0] (rows=2000 width=87)
                         default@srcpart_date_n9,srcpart_date_n9,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
-                Dynamic Partitioning Event Operator [EVENT_21] (rows=309 width=87)
-                  Group By Operator [GBY_20] (rows=309 width=87)
+                Dynamic Partitioning Event Operator [EVENT_21] (rows=316 width=87)
+                  Group By Operator [GBY_20] (rows=316 width=87)
                     Output:["_col0"],keys:_col0
                     Select Operator [SEL_19] (rows=2000 width=87)
                       Output:["_col0"]
@@ -434,11 +434,11 @@ Stage-0
                       <-Reducer 4 [BROADCAST_EDGE] llap
                         BROADCAST [RS_44]
                           Group By Operator [GBY_41] (rows=1 width=552)
-                            Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=309)"]
+                            Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=316)"]
                           <-Map 1 [CUSTOM_SIMPLE_EDGE] llap
                             PARTITION_ONLY_SHUFFLE [RS_40]
                               Group By Operator [GBY_39] (rows=1 width=552)
-                                Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=309)"]
+                                Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=316)"]
                                 Select Operator [SEL_38] (rows=2000 width=87)
                                   Output:["_col0"]
                                    Please refer to the previous Select Operator [SEL_2]


[10/14] hive git commit: HIVE-18079 : Statistics: Allow HyperLogLog to be merged to the lowest-common-denominator bit-size (Gopal V via Prasanth J)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/13960aa9/ql/src/test/results/clientpositive/llap/explainanalyze_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/explainanalyze_2.q.out b/ql/src/test/results/clientpositive/llap/explainanalyze_2.q.out
index c1cc477..6ec4d00 100644
--- a/ql/src/test/results/clientpositive/llap/explainanalyze_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainanalyze_2.q.out
@@ -41,134 +41,129 @@ POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Union 2 (CONTAINS)
+Map 12 <- Union 10 (CONTAINS)
 Map 13 <- Union 14 (CONTAINS)
 Map 16 <- Union 14 (CONTAINS)
-Map 8 <- Union 2 (CONTAINS)
-Reducer 10 <- Map 9 (SIMPLE_EDGE), Reducer 15 (SIMPLE_EDGE)
-Reducer 11 <- Map 17 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Map 9 <- Union 10 (CONTAINS)
+Reducer 11 <- Union 10 (SIMPLE_EDGE)
 Reducer 15 <- Union 14 (SIMPLE_EDGE)
-Reducer 3 <- Union 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 12 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 7 <- Union 6 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 11 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 5 <- Union 4 (SIMPLE_EDGE)
+Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 15 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE), Union 4 (CONTAINS)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 7 llap
+      Reducer 5 llap
       File Output Operator [FS_56]
-        Group By Operator [GBY_54] (rows=32/15 width=177)
+        Group By Operator [GBY_54] (rows=48/15 width=177)
           Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-        <-Union 6 [SIMPLE_EDGE]
-          <-Reducer 11 [CONTAINS] llap
+        <-Union 4 [SIMPLE_EDGE]
+          <-Reducer 3 [CONTAINS] llap
             Reduce Output Operator [RS_53]
               PartitionCols:_col0, _col1
-              Select Operator [SEL_49] (rows=67/61 width=177)
+              Select Operator [SEL_24] (rows=66/61 width=177)
                 Output:["_col0","_col1"]
-                Merge Join Operator [MERGEJOIN_82] (rows=67/61 width=177)
-                  Conds:RS_46._col2=RS_47._col0(Inner),Output:["_col1","_col2"]
-                <-Map 17 [SIMPLE_EDGE] llap
-                  SHUFFLE [RS_47]
-                    PartitionCols:_col0
-                    Select Operator [SEL_42] (rows=500/500 width=87)
-                      Output:["_col0"]
-                      Filter Operator [FIL_78] (rows=500/500 width=87)
-                        predicate:key is not null
-                        TableScan [TS_40] (rows=500/500 width=87)
-                          default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
-                <-Reducer 10 [SIMPLE_EDGE] llap
-                  SHUFFLE [RS_46]
+                Merge Join Operator [MERGEJOIN_79] (rows=66/61 width=177)
+                  Conds:RS_21._col2=RS_22._col1(Inner),Output:["_col1","_col4"]
+                <-Reducer 11 [SIMPLE_EDGE] llap
+                  SHUFFLE [RS_22]
+                    PartitionCols:_col1
+                    Select Operator [SEL_17] (rows=525/319 width=178)
+                      Output:["_col1"]
+                      Group By Operator [GBY_16] (rows=525/319 width=178)
+                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                      <-Union 10 [SIMPLE_EDGE]
+                        <-Map 12 [CONTAINS] llap
+                          Reduce Output Operator [RS_15]
+                            PartitionCols:_col1, _col0
+                            Select Operator [SEL_11] (rows=500/500 width=178)
+                              Output:["_col0","_col1"]
+                              Filter Operator [FIL_72] (rows=500/500 width=178)
+                                predicate:value is not null
+                                TableScan [TS_9] (rows=500/500 width=178)
+                                  Output:["key","value"]
+                        <-Map 9 [CONTAINS] llap
+                          Reduce Output Operator [RS_15]
+                            PartitionCols:_col1, _col0
+                            Select Operator [SEL_8] (rows=25/25 width=175)
+                              Output:["_col0","_col1"]
+                              Filter Operator [FIL_71] (rows=25/25 width=175)
+                                predicate:value is not null
+                                TableScan [TS_6] (rows=25/25 width=175)
+                                  Output:["key","value"]
+                <-Reducer 2 [SIMPLE_EDGE] llap
+                  SHUFFLE [RS_21]
                     PartitionCols:_col2
-                    Merge Join Operator [MERGEJOIN_81] (rows=42/52 width=177)
-                      Conds:RS_43._col1=RS_44._col1(Inner),Output:["_col1","_col2"]
-                    <-Map 9 [SIMPLE_EDGE] llap
-                      SHUFFLE [RS_44]
-                        PartitionCols:_col1
-                        Select Operator [SEL_14] (rows=25/25 width=175)
+                    Merge Join Operator [MERGEJOIN_77] (rows=39/37 width=175)
+                      Conds:RS_18._col0=RS_19._col0(Inner),Output:["_col1","_col2"]
+                    <-Map 1 [SIMPLE_EDGE] llap
+                      SHUFFLE [RS_18]
+                        PartitionCols:_col0
+                        Select Operator [SEL_2] (rows=500/500 width=87)
+                          Output:["_col0"]
+                          Filter Operator [FIL_69] (rows=500/500 width=87)
+                            predicate:key is not null
+                            TableScan [TS_0] (rows=500/500 width=87)
+                              default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
+                    <-Map 8 [SIMPLE_EDGE] llap
+                      SHUFFLE [RS_19]
+                        PartitionCols:_col0
+                        Select Operator [SEL_5] (rows=25/25 width=175)
                           Output:["_col0","_col1"]
-                          Filter Operator [FIL_73] (rows=25/25 width=175)
+                          Filter Operator [FIL_70] (rows=25/25 width=175)
                             predicate:(key is not null and value is not null)
-                            TableScan [TS_12] (rows=25/25 width=175)
+                            TableScan [TS_3] (rows=25/25 width=175)
                               default@src1,x,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                    <-Reducer 15 [SIMPLE_EDGE] llap
-                      SHUFFLE [RS_43]
-                        PartitionCols:_col1
-                        Select Operator [SEL_36] (rows=525/319 width=178)
-                          Output:["_col1"]
-                          Group By Operator [GBY_35] (rows=525/319 width=178)
-                            Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                          <-Union 14 [SIMPLE_EDGE]
-                            <-Map 13 [CONTAINS] llap
-                              Reduce Output Operator [RS_34]
-                                PartitionCols:_col1, _col0
-                                Select Operator [SEL_27] (rows=25/25 width=175)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_75] (rows=25/25 width=175)
-                                    predicate:value is not null
-                                    TableScan [TS_25] (rows=25/25 width=175)
-                                      Output:["key","value"]
-                            <-Map 16 [CONTAINS] llap
-                              Reduce Output Operator [RS_34]
-                                PartitionCols:_col1, _col0
-                                Select Operator [SEL_30] (rows=500/500 width=178)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_76] (rows=500/500 width=178)
-                                    predicate:value is not null
-                                    TableScan [TS_28] (rows=500/500 width=178)
-                                      Output:["key","value"]
-          <-Reducer 5 [CONTAINS] llap
+          <-Reducer 7 [CONTAINS] llap
             Reduce Output Operator [RS_53]
               PartitionCols:_col0, _col1
-              Select Operator [SEL_24] (rows=67/61 width=177)
+              Select Operator [SEL_49] (rows=66/61 width=177)
                 Output:["_col0","_col1"]
-                Merge Join Operator [MERGEJOIN_80] (rows=67/61 width=177)
-                  Conds:RS_21._col2=RS_22._col0(Inner),Output:["_col1","_col2"]
-                <-Map 12 [SIMPLE_EDGE] llap
-                  SHUFFLE [RS_22]
-                    PartitionCols:_col0
-                    Select Operator [SEL_17] (rows=500/500 width=87)
-                      Output:["_col0"]
-                      Filter Operator [FIL_74] (rows=500/500 width=87)
-                        predicate:key is not null
-                        TableScan [TS_15] (rows=500/500 width=87)
-                          default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
-                <-Reducer 4 [SIMPLE_EDGE] llap
-                  SHUFFLE [RS_21]
+                Merge Join Operator [MERGEJOIN_80] (rows=66/61 width=177)
+                  Conds:RS_46._col2=RS_47._col1(Inner),Output:["_col1","_col4"]
+                <-Reducer 15 [SIMPLE_EDGE] llap
+                  SHUFFLE [RS_47]
+                    PartitionCols:_col1
+                    Select Operator [SEL_42] (rows=525/319 width=178)
+                      Output:["_col1"]
+                      Group By Operator [GBY_41] (rows=525/319 width=178)
+                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                      <-Union 14 [SIMPLE_EDGE]
+                        <-Map 13 [CONTAINS] llap
+                          Reduce Output Operator [RS_40]
+                            PartitionCols:_col1, _col0
+                            Select Operator [SEL_33] (rows=25/25 width=175)
+                              Output:["_col0","_col1"]
+                              Filter Operator [FIL_75] (rows=25/25 width=175)
+                                predicate:value is not null
+                                TableScan [TS_31] (rows=25/25 width=175)
+                                  Output:["key","value"]
+                        <-Map 16 [CONTAINS] llap
+                          Reduce Output Operator [RS_40]
+                            PartitionCols:_col1, _col0
+                            Select Operator [SEL_36] (rows=500/500 width=178)
+                              Output:["_col0","_col1"]
+                              Filter Operator [FIL_76] (rows=500/500 width=178)
+                                predicate:value is not null
+                                TableScan [TS_34] (rows=500/500 width=178)
+                                  Output:["key","value"]
+                <-Reducer 6 [SIMPLE_EDGE] llap
+                  SHUFFLE [RS_46]
                     PartitionCols:_col2
-                    Merge Join Operator [MERGEJOIN_79] (rows=42/52 width=177)
-                      Conds:RS_18._col1=RS_19._col1(Inner),Output:["_col1","_col2"]
-                    <-Map 9 [SIMPLE_EDGE] llap
-                      SHUFFLE [RS_19]
-                        PartitionCols:_col1
-                         Please refer to the previous Select Operator [SEL_14]
-                    <-Reducer 3 [SIMPLE_EDGE] llap
-                      SHUFFLE [RS_18]
-                        PartitionCols:_col1
-                        Select Operator [SEL_11] (rows=525/319 width=178)
-                          Output:["_col1"]
-                          Group By Operator [GBY_10] (rows=525/319 width=178)
-                            Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                          <-Union 2 [SIMPLE_EDGE]
-                            <-Map 1 [CONTAINS] llap
-                              Reduce Output Operator [RS_9]
-                                PartitionCols:_col1, _col0
-                                Select Operator [SEL_2] (rows=25/25 width=175)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_71] (rows=25/25 width=175)
-                                    predicate:value is not null
-                                    TableScan [TS_0] (rows=25/25 width=175)
-                                      Output:["key","value"]
-                            <-Map 8 [CONTAINS] llap
-                              Reduce Output Operator [RS_9]
-                                PartitionCols:_col1, _col0
-                                Select Operator [SEL_5] (rows=500/500 width=178)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_72] (rows=500/500 width=178)
-                                    predicate:value is not null
-                                    TableScan [TS_3] (rows=500/500 width=178)
-                                      Output:["key","value"]
+                    Merge Join Operator [MERGEJOIN_78] (rows=39/37 width=175)
+                      Conds:RS_43._col0=RS_44._col0(Inner),Output:["_col1","_col2"]
+                    <-Map 1 [SIMPLE_EDGE] llap
+                      SHUFFLE [RS_43]
+                        PartitionCols:_col0
+                         Please refer to the previous Select Operator [SEL_2]
+                    <-Map 8 [SIMPLE_EDGE] llap
+                      SHUFFLE [RS_44]
+                        PartitionCols:_col0
+                         Please refer to the previous Select Operator [SEL_5]
 
 PREHOOK: query: SELECT x.key, y.value
 FROM src1 x JOIN src y ON (x.key = y.key) 
@@ -229,249 +224,234 @@ POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Union 2 (CONTAINS)
-Map 10 <- Union 2 (CONTAINS)
-Map 17 <- Union 18 (CONTAINS)
-Map 22 <- Union 18 (CONTAINS)
-Map 23 <- Union 20 (CONTAINS)
-Map 25 <- Union 26 (CONTAINS)
-Map 32 <- Union 26 (CONTAINS)
-Map 33 <- Union 28 (CONTAINS)
-Map 34 <- Union 30 (CONTAINS)
-Reducer 12 <- Map 11 (SIMPLE_EDGE), Reducer 21 (SIMPLE_EDGE)
-Reducer 13 <- Map 24 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 14 <- Map 11 (SIMPLE_EDGE), Reducer 31 (SIMPLE_EDGE)
-Reducer 15 <- Map 16 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE), Union 8 (CONTAINS)
-Reducer 19 <- Union 18 (SIMPLE_EDGE), Union 20 (CONTAINS)
-Reducer 21 <- Union 20 (SIMPLE_EDGE)
+Map 12 <- Union 13 (CONTAINS)
+Map 15 <- Union 13 (CONTAINS)
+Map 16 <- Union 17 (CONTAINS)
+Map 21 <- Union 17 (CONTAINS)
+Map 22 <- Union 19 (CONTAINS)
+Map 23 <- Union 24 (CONTAINS)
+Map 30 <- Union 24 (CONTAINS)
+Map 31 <- Union 26 (CONTAINS)
+Map 32 <- Union 28 (CONTAINS)
+Reducer 10 <- Reducer 20 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 14 <- Union 13 (SIMPLE_EDGE)
+Reducer 18 <- Union 17 (SIMPLE_EDGE), Union 19 (CONTAINS)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
+Reducer 20 <- Union 19 (SIMPLE_EDGE)
+Reducer 25 <- Union 24 (SIMPLE_EDGE), Union 26 (CONTAINS)
 Reducer 27 <- Union 26 (SIMPLE_EDGE), Union 28 (CONTAINS)
-Reducer 29 <- Union 28 (SIMPLE_EDGE), Union 30 (CONTAINS)
-Reducer 3 <- Union 2 (SIMPLE_EDGE)
-Reducer 31 <- Union 30 (SIMPLE_EDGE)
-Reducer 4 <- Map 11 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 16 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 7 <- Union 6 (SIMPLE_EDGE), Union 8 (CONTAINS)
-Reducer 9 <- Union 8 (SIMPLE_EDGE)
+Reducer 29 <- Union 28 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 14 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 5 <- Union 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 7 <- Union 6 (SIMPLE_EDGE)
+Reducer 8 <- Reducer 2 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 9 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 9 llap
+      Reducer 7 llap
       File Output Operator [FS_114]
-        Group By Operator [GBY_112] (rows=398/15 width=177)
+        Group By Operator [GBY_112] (rows=384/15 width=177)
           Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-        <-Union 8 [SIMPLE_EDGE]
-          <-Reducer 15 [CONTAINS] llap
-            Reduce Output Operator [RS_111]
-              PartitionCols:_col0, _col1
-              Select Operator [SEL_107] (rows=199/61 width=177)
-                Output:["_col0","_col1"]
-                Merge Join Operator [MERGEJOIN_162] (rows=199/61 width=177)
-                  Conds:RS_104._col2=RS_105._col0(Inner),Output:["_col2","_col5"]
-                <-Map 16 [SIMPLE_EDGE] llap
-                  SHUFFLE [RS_105]
-                    PartitionCols:_col0
-                    Select Operator [SEL_17] (rows=500/500 width=178)
-                      Output:["_col0","_col1"]
-                      Filter Operator [FIL_145] (rows=500/500 width=178)
-                        predicate:key is not null
-                        TableScan [TS_15] (rows=500/500 width=178)
-                          default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                <-Reducer 14 [SIMPLE_EDGE] llap
-                  SHUFFLE [RS_104]
-                    PartitionCols:_col2
-                    Merge Join Operator [MERGEJOIN_161] (rows=123/52 width=86)
-                      Conds:RS_101._col1=RS_102._col1(Inner),Output:["_col2"]
-                    <-Map 11 [SIMPLE_EDGE] llap
-                      SHUFFLE [RS_102]
-                        PartitionCols:_col1
-                        Select Operator [SEL_14] (rows=25/25 width=175)
-                          Output:["_col0","_col1"]
-                          Filter Operator [FIL_144] (rows=25/25 width=175)
-                            predicate:(key is not null and value is not null)
-                            TableScan [TS_12] (rows=25/25 width=175)
-                              default@src1,x,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                    <-Reducer 31 [SIMPLE_EDGE] llap
-                      SHUFFLE [RS_101]
-                        PartitionCols:_col1
-                        Select Operator [SEL_94] (rows=1525/319 width=178)
-                          Output:["_col1"]
-                          Group By Operator [GBY_93] (rows=1525/319 width=178)
-                            Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                          <-Union 30 [SIMPLE_EDGE]
-                            <-Map 34 [CONTAINS] llap
-                              Reduce Output Operator [RS_92]
-                                PartitionCols:_col1, _col0
-                                Select Operator [SEL_88] (rows=500/500 width=178)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_154] (rows=500/500 width=178)
-                                    predicate:value is not null
-                                    TableScan [TS_86] (rows=500/500 width=178)
-                                      Output:["key","value"]
-                            <-Reducer 29 [CONTAINS] llap
-                              Reduce Output Operator [RS_92]
-                                PartitionCols:_col1, _col0
-                                Select Operator [SEL_85] (rows=1025/319 width=178)
-                                  Output:["_col0","_col1"]
-                                  Group By Operator [GBY_84] (rows=1025/319 width=178)
-                                    Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                  <-Union 28 [SIMPLE_EDGE]
-                                    <-Map 33 [CONTAINS] llap
-                                      Reduce Output Operator [RS_83]
-                                        PartitionCols:_col1, _col0
-                                        Select Operator [SEL_79] (rows=500/500 width=178)
-                                          Output:["_col0","_col1"]
-                                          Filter Operator [FIL_153] (rows=500/500 width=178)
-                                            predicate:value is not null
-                                            TableScan [TS_77] (rows=500/500 width=178)
-                                              Output:["key","value"]
-                                    <-Reducer 27 [CONTAINS] llap
-                                      Reduce Output Operator [RS_83]
-                                        PartitionCols:_col1, _col0
-                                        Select Operator [SEL_76] (rows=525/319 width=178)
-                                          Output:["_col0","_col1"]
-                                          Group By Operator [GBY_75] (rows=525/319 width=178)
-                                            Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                          <-Union 26 [SIMPLE_EDGE]
-                                            <-Map 25 [CONTAINS] llap
-                                              Reduce Output Operator [RS_74]
-                                                PartitionCols:_col1, _col0
-                                                Select Operator [SEL_67] (rows=25/25 width=175)
-                                                  Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_151] (rows=25/25 width=175)
-                                                    predicate:value is not null
-                                                    TableScan [TS_65] (rows=25/25 width=175)
-                                                      Output:["key","value"]
-                                            <-Map 32 [CONTAINS] llap
-                                              Reduce Output Operator [RS_74]
-                                                PartitionCols:_col1, _col0
-                                                Select Operator [SEL_70] (rows=500/500 width=178)
-                                                  Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_152] (rows=500/500 width=178)
-                                                    predicate:value is not null
-                                                    TableScan [TS_68] (rows=500/500 width=178)
-                                                      Output:["key","value"]
-          <-Reducer 7 [CONTAINS] llap
+        <-Union 6 [SIMPLE_EDGE]
+          <-Reducer 5 [CONTAINS] llap
             Reduce Output Operator [RS_111]
               PartitionCols:_col0, _col1
-              Group By Operator [GBY_63] (rows=199/15 width=177)
+              Group By Operator [GBY_63] (rows=196/15 width=177)
                 Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-              <-Union 6 [SIMPLE_EDGE]
-                <-Reducer 13 [CONTAINS] llap
+              <-Union 4 [SIMPLE_EDGE]
+                <-Reducer 10 [CONTAINS] llap
                   Reduce Output Operator [RS_62]
                     PartitionCols:_col0, _col1
-                    Select Operator [SEL_58] (rows=132/61 width=177)
+                    Select Operator [SEL_58] (rows=130/61 width=177)
                       Output:["_col0","_col1"]
-                      Merge Join Operator [MERGEJOIN_160] (rows=132/61 width=177)
-                        Conds:RS_55._col2=RS_56._col0(Inner),Output:["_col2","_col5"]
-                      <-Map 24 [SIMPLE_EDGE] llap
+                      Merge Join Operator [MERGEJOIN_155] (rows=130/61 width=177)
+                        Conds:RS_55._col3=RS_56._col1(Inner),Output:["_col1","_col2"]
+                      <-Reducer 20 [SIMPLE_EDGE] llap
                         SHUFFLE [RS_56]
-                          PartitionCols:_col0
-                          Select Operator [SEL_51] (rows=500/500 width=178)
-                            Output:["_col0","_col1"]
-                            Filter Operator [FIL_150] (rows=500/500 width=178)
-                              predicate:key is not null
-                              TableScan [TS_49] (rows=500/500 width=178)
-                                default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                      <-Reducer 12 [SIMPLE_EDGE] llap
+                          PartitionCols:_col1
+                          Select Operator [SEL_51] (rows=1025/319 width=178)
+                            Output:["_col1"]
+                            Group By Operator [GBY_50] (rows=1025/319 width=178)
+                              Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                            <-Union 19 [SIMPLE_EDGE]
+                              <-Map 22 [CONTAINS] llap
+                                Reduce Output Operator [RS_49]
+                                  PartitionCols:_col1, _col0
+                                  Select Operator [SEL_45] (rows=500/500 width=178)
+                                    Output:["_col0","_col1"]
+                                    Filter Operator [FIL_144] (rows=500/500 width=178)
+                                      predicate:value is not null
+                                      TableScan [TS_43] (rows=500/500 width=178)
+                                        Output:["key","value"]
+                              <-Reducer 18 [CONTAINS] llap
+                                Reduce Output Operator [RS_49]
+                                  PartitionCols:_col1, _col0
+                                  Select Operator [SEL_42] (rows=525/319 width=178)
+                                    Output:["_col0","_col1"]
+                                    Group By Operator [GBY_41] (rows=525/319 width=178)
+                                      Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                    <-Union 17 [SIMPLE_EDGE]
+                                      <-Map 16 [CONTAINS] llap
+                                        Reduce Output Operator [RS_40]
+                                          PartitionCols:_col1, _col0
+                                          Select Operator [SEL_33] (rows=25/25 width=175)
+                                            Output:["_col0","_col1"]
+                                            Filter Operator [FIL_142] (rows=25/25 width=175)
+                                              predicate:value is not null
+                                              TableScan [TS_31] (rows=25/25 width=175)
+                                                Output:["key","value"]
+                                      <-Map 21 [CONTAINS] llap
+                                        Reduce Output Operator [RS_40]
+                                          PartitionCols:_col1, _col0
+                                          Select Operator [SEL_36] (rows=500/500 width=178)
+                                            Output:["_col0","_col1"]
+                                            Filter Operator [FIL_143] (rows=500/500 width=178)
+                                              predicate:value is not null
+                                              TableScan [TS_34] (rows=500/500 width=178)
+                                                Output:["key","value"]
+                      <-Reducer 9 [SIMPLE_EDGE] llap
                         SHUFFLE [RS_55]
-                          PartitionCols:_col2
-                          Merge Join Operator [MERGEJOIN_159] (rows=82/52 width=86)
-                            Conds:RS_52._col1=RS_53._col1(Inner),Output:["_col2"]
+                          PartitionCols:_col3
+                          Merge Join Operator [MERGEJOIN_152] (rows=39/37 width=266)
+                            Conds:RS_52._col0=RS_53._col0(Inner),Output:["_col1","_col2","_col3"]
+                          <-Map 1 [SIMPLE_EDGE] llap
+                            SHUFFLE [RS_52]
+                              PartitionCols:_col0
+                              Select Operator [SEL_2] (rows=500/500 width=178)
+                                Output:["_col0","_col1"]
+                                Filter Operator [FIL_136] (rows=500/500 width=178)
+                                  predicate:key is not null
+                                  TableScan [TS_0] (rows=500/500 width=178)
+                                    default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
                           <-Map 11 [SIMPLE_EDGE] llap
                             SHUFFLE [RS_53]
-                              PartitionCols:_col1
-                               Please refer to the previous Select Operator [SEL_14]
-                          <-Reducer 21 [SIMPLE_EDGE] llap
-                            SHUFFLE [RS_52]
-                              PartitionCols:_col1
-                              Select Operator [SEL_45] (rows=1025/319 width=178)
-                                Output:["_col1"]
-                                Group By Operator [GBY_44] (rows=1025/319 width=178)
-                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                <-Union 20 [SIMPLE_EDGE]
-                                  <-Map 23 [CONTAINS] llap
-                                    Reduce Output Operator [RS_43]
-                                      PartitionCols:_col1, _col0
-                                      Select Operator [SEL_39] (rows=500/500 width=178)
-                                        Output:["_col0","_col1"]
-                                        Filter Operator [FIL_148] (rows=500/500 width=178)
-                                          predicate:value is not null
-                                          TableScan [TS_37] (rows=500/500 width=178)
-                                            Output:["key","value"]
-                                  <-Reducer 19 [CONTAINS] llap
-                                    Reduce Output Operator [RS_43]
-                                      PartitionCols:_col1, _col0
-                                      Select Operator [SEL_36] (rows=525/319 width=178)
-                                        Output:["_col0","_col1"]
-                                        Group By Operator [GBY_35] (rows=525/319 width=178)
-                                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                        <-Union 18 [SIMPLE_EDGE]
-                                          <-Map 17 [CONTAINS] llap
-                                            Reduce Output Operator [RS_34]
-                                              PartitionCols:_col1, _col0
-                                              Select Operator [SEL_27] (rows=25/25 width=175)
-                                                Output:["_col0","_col1"]
-                                                Filter Operator [FIL_146] (rows=25/25 width=175)
-                                                  predicate:value is not null
-                                                  TableScan [TS_25] (rows=25/25 width=175)
-                                                    Output:["key","value"]
-                                          <-Map 22 [CONTAINS] llap
-                                            Reduce Output Operator [RS_34]
-                                              PartitionCols:_col1, _col0
-                                              Select Operator [SEL_30] (rows=500/500 width=178)
-                                                Output:["_col0","_col1"]
-                                                Filter Operator [FIL_147] (rows=500/500 width=178)
-                                                  predicate:value is not null
-                                                  TableScan [TS_28] (rows=500/500 width=178)
-                                                    Output:["key","value"]
-                <-Reducer 5 [CONTAINS] llap
+                              PartitionCols:_col0
+                              Select Operator [SEL_5] (rows=25/25 width=175)
+                                Output:["_col0","_col1"]
+                                Filter Operator [FIL_137] (rows=25/25 width=175)
+                                  predicate:(key is not null and value is not null)
+                                  TableScan [TS_3] (rows=25/25 width=175)
+                                    default@src1,x,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                <-Reducer 3 [CONTAINS] llap
                   Reduce Output Operator [RS_62]
                     PartitionCols:_col0, _col1
-                    Select Operator [SEL_24] (rows=67/61 width=177)
+                    Select Operator [SEL_24] (rows=66/61 width=177)
                       Output:["_col0","_col1"]
-                      Merge Join Operator [MERGEJOIN_158] (rows=67/61 width=177)
-                        Conds:RS_21._col2=RS_22._col0(Inner),Output:["_col2","_col5"]
-                      <-Map 16 [SIMPLE_EDGE] llap
-                        SHUFFLE [RS_22]
-                          PartitionCols:_col0
-                           Please refer to the previous Select Operator [SEL_17]
-                      <-Reducer 4 [SIMPLE_EDGE] llap
+                      Merge Join Operator [MERGEJOIN_154] (rows=66/61 width=177)
+                        Conds:RS_21._col3=RS_22._col1(Inner),Output:["_col1","_col2"]
+                      <-Reducer 2 [SIMPLE_EDGE] llap
                         SHUFFLE [RS_21]
-                          PartitionCols:_col2
-                          Merge Join Operator [MERGEJOIN_157] (rows=42/52 width=86)
-                            Conds:RS_18._col1=RS_19._col1(Inner),Output:["_col2"]
+                          PartitionCols:_col3
+                          Merge Join Operator [MERGEJOIN_151] (rows=39/37 width=266)
+                            Conds:RS_18._col0=RS_19._col0(Inner),Output:["_col1","_col2","_col3"]
+                          <-Map 1 [SIMPLE_EDGE] llap
+                            SHUFFLE [RS_18]
+                              PartitionCols:_col0
+                               Please refer to the previous Select Operator [SEL_2]
                           <-Map 11 [SIMPLE_EDGE] llap
                             SHUFFLE [RS_19]
-                              PartitionCols:_col1
-                               Please refer to the previous Select Operator [SEL_14]
-                          <-Reducer 3 [SIMPLE_EDGE] llap
-                            SHUFFLE [RS_18]
-                              PartitionCols:_col1
-                              Select Operator [SEL_11] (rows=525/319 width=178)
-                                Output:["_col1"]
-                                Group By Operator [GBY_10] (rows=525/319 width=178)
-                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                <-Union 2 [SIMPLE_EDGE]
-                                  <-Map 1 [CONTAINS] llap
-                                    Reduce Output Operator [RS_9]
-                                      PartitionCols:_col1, _col0
-                                      Select Operator [SEL_2] (rows=25/25 width=175)
-                                        Output:["_col0","_col1"]
-                                        Filter Operator [FIL_142] (rows=25/25 width=175)
-                                          predicate:value is not null
-                                          TableScan [TS_0] (rows=25/25 width=175)
-                                            Output:["key","value"]
-                                  <-Map 10 [CONTAINS] llap
-                                    Reduce Output Operator [RS_9]
-                                      PartitionCols:_col1, _col0
-                                      Select Operator [SEL_5] (rows=500/500 width=178)
-                                        Output:["_col0","_col1"]
-                                        Filter Operator [FIL_143] (rows=500/500 width=178)
-                                          predicate:value is not null
-                                          TableScan [TS_3] (rows=500/500 width=178)
-                                            Output:["key","value"]
+                              PartitionCols:_col0
+                               Please refer to the previous Select Operator [SEL_5]
+                      <-Reducer 14 [SIMPLE_EDGE] llap
+                        SHUFFLE [RS_22]
+                          PartitionCols:_col1
+                          Select Operator [SEL_17] (rows=525/319 width=178)
+                            Output:["_col1"]
+                            Group By Operator [GBY_16] (rows=525/319 width=178)
+                              Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                            <-Union 13 [SIMPLE_EDGE]
+                              <-Map 12 [CONTAINS] llap
+                                Reduce Output Operator [RS_15]
+                                  PartitionCols:_col1, _col0
+                                  Select Operator [SEL_8] (rows=25/25 width=175)
+                                    Output:["_col0","_col1"]
+                                    Filter Operator [FIL_138] (rows=25/25 width=175)
+                                      predicate:value is not null
+                                      TableScan [TS_6] (rows=25/25 width=175)
+                                        Output:["key","value"]
+                              <-Map 15 [CONTAINS] llap
+                                Reduce Output Operator [RS_15]
+                                  PartitionCols:_col1, _col0
+                                  Select Operator [SEL_11] (rows=500/500 width=178)
+                                    Output:["_col0","_col1"]
+                                    Filter Operator [FIL_139] (rows=500/500 width=178)
+                                      predicate:value is not null
+                                      TableScan [TS_9] (rows=500/500 width=178)
+                                        Output:["key","value"]
+          <-Reducer 8 [CONTAINS] llap
+            Reduce Output Operator [RS_111]
+              PartitionCols:_col0, _col1
+              Select Operator [SEL_107] (rows=193/61 width=177)
+                Output:["_col0","_col1"]
+                Merge Join Operator [MERGEJOIN_156] (rows=193/61 width=177)
+                  Conds:RS_104._col3=RS_105._col1(Inner),Output:["_col1","_col2"]
+                <-Reducer 2 [SIMPLE_EDGE] llap
+                  SHUFFLE [RS_104]
+                    PartitionCols:_col3
+                     Please refer to the previous Merge Join Operator [MERGEJOIN_151]
+                <-Reducer 29 [SIMPLE_EDGE] llap
+                  SHUFFLE [RS_105]
+                    PartitionCols:_col1
+                    Select Operator [SEL_100] (rows=1525/319 width=178)
+                      Output:["_col1"]
+                      Group By Operator [GBY_99] (rows=1525/319 width=178)
+                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                      <-Union 28 [SIMPLE_EDGE]
+                        <-Map 32 [CONTAINS] llap
+                          Reduce Output Operator [RS_98]
+                            PartitionCols:_col1, _col0
+                            Select Operator [SEL_94] (rows=500/500 width=178)
+                              Output:["_col0","_col1"]
+                              Filter Operator [FIL_150] (rows=500/500 width=178)
+                                predicate:value is not null
+                                TableScan [TS_92] (rows=500/500 width=178)
+                                  Output:["key","value"]
+                        <-Reducer 27 [CONTAINS] llap
+                          Reduce Output Operator [RS_98]
+                            PartitionCols:_col1, _col0
+                            Select Operator [SEL_91] (rows=1025/319 width=178)
+                              Output:["_col0","_col1"]
+                              Group By Operator [GBY_90] (rows=1025/319 width=178)
+                                Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                              <-Union 26 [SIMPLE_EDGE]
+                                <-Map 31 [CONTAINS] llap
+                                  Reduce Output Operator [RS_89]
+                                    PartitionCols:_col1, _col0
+                                    Select Operator [SEL_85] (rows=500/500 width=178)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_149] (rows=500/500 width=178)
+                                        predicate:value is not null
+                                        TableScan [TS_83] (rows=500/500 width=178)
+                                          Output:["key","value"]
+                                <-Reducer 25 [CONTAINS] llap
+                                  Reduce Output Operator [RS_89]
+                                    PartitionCols:_col1, _col0
+                                    Select Operator [SEL_82] (rows=525/319 width=178)
+                                      Output:["_col0","_col1"]
+                                      Group By Operator [GBY_81] (rows=525/319 width=178)
+                                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                      <-Union 24 [SIMPLE_EDGE]
+                                        <-Map 23 [CONTAINS] llap
+                                          Reduce Output Operator [RS_80]
+                                            PartitionCols:_col1, _col0
+                                            Select Operator [SEL_73] (rows=25/25 width=175)
+                                              Output:["_col0","_col1"]
+                                              Filter Operator [FIL_147] (rows=25/25 width=175)
+                                                predicate:value is not null
+                                                TableScan [TS_71] (rows=25/25 width=175)
+                                                  Output:["key","value"]
+                                        <-Map 30 [CONTAINS] llap
+                                          Reduce Output Operator [RS_80]
+                                            PartitionCols:_col1, _col0
+                                            Select Operator [SEL_76] (rows=500/500 width=178)
+                                              Output:["_col0","_col1"]
+                                              Filter Operator [FIL_148] (rows=500/500 width=178)
+                                                predicate:value is not null
+                                                TableScan [TS_74] (rows=500/500 width=178)
+                                                  Output:["key","value"]
 
 PREHOOK: query: CREATE TABLE srcbucket_mapjoin_n11(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE
 PREHOOK: type: CREATETABLE
@@ -629,7 +609,7 @@ Stage-0
     Stage-1
       Map 1 llap
       File Output Operator [FS_10]
-        Merge Join Operator [MERGEJOIN_15] (rows=382/480 width=95)
+        Merge Join Operator [MERGEJOIN_15] (rows=401/480 width=95)
           Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0","_col1"]
         <-Select Operator [SEL_5] (rows=242/242 width=4)
             Output:["_col0"]
@@ -664,41 +644,48 @@ Plan optimized by CBO.
 
 Vertex dependency in root stage
 Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 2 llap
+      Reducer 3 llap
       File Output Operator [FS_16]
-        Merge Join Operator [MERGEJOIN_27] (rows=604/1166 width=95)
-          Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col0","_col1"]
-        <-Map 1 [SIMPLE_EDGE] llap
-          SHUFFLE [RS_12]
-            PartitionCols:_col1
-            Merge Join Operator [MERGEJOIN_25] (rows=382/480 width=95)
-              Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0","_col1"]
-            <-Select Operator [SEL_5] (rows=242/242 width=4)
-                Output:["_col0"]
-                Filter Operator [FIL_23] (rows=242/242 width=4)
-                  predicate:key is not null
-                  TableScan [TS_3] (rows=242/242 width=4)
-                    default@tab_n6,s3,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
-            <-Select Operator [SEL_2] (rows=242/242 width=95)
-                Output:["_col0","_col1"]
-                Filter Operator [FIL_22] (rows=242/242 width=95)
-                  predicate:(key is not null and value is not null)
-                  TableScan [TS_0] (rows=242/242 width=95)
-                    default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-        <-Map 4 [SIMPLE_EDGE] llap
+        Merge Join Operator [MERGEJOIN_26] (rows=633/1166 width=95)
+          Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col0","_col1"]
+        <-Map 5 [SIMPLE_EDGE] llap
           SHUFFLE [RS_13]
             PartitionCols:_col0
-            Select Operator [SEL_8] (rows=242/242 width=91)
+            Select Operator [SEL_8] (rows=242/242 width=4)
               Output:["_col0"]
-              Filter Operator [FIL_24] (rows=242/242 width=91)
-                predicate:value is not null
-                TableScan [TS_6] (rows=242/242 width=91)
-                  default@tab_n6,s2,Tbl:COMPLETE,Col:COMPLETE,Output:["value"]
+              Filter Operator [FIL_24] (rows=242/242 width=4)
+                predicate:key is not null
+                TableScan [TS_6] (rows=242/242 width=4)
+                  default@tab_n6,s3,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
+        <-Reducer 2 [SIMPLE_EDGE] llap
+          SHUFFLE [RS_12]
+            PartitionCols:_col0
+            Merge Join Operator [MERGEJOIN_25] (rows=382/480 width=95)
+              Conds:RS_9._col1=RS_10._col0(Inner),Output:["_col0","_col1"]
+            <-Map 1 [SIMPLE_EDGE] llap
+              SHUFFLE [RS_9]
+                PartitionCols:_col1
+                Select Operator [SEL_2] (rows=242/242 width=95)
+                  Output:["_col0","_col1"]
+                  Filter Operator [FIL_22] (rows=242/242 width=95)
+                    predicate:(key is not null and value is not null)
+                    TableScan [TS_0] (rows=242/242 width=95)
+                      default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+            <-Map 4 [SIMPLE_EDGE] llap
+              SHUFFLE [RS_10]
+                PartitionCols:_col0
+                Select Operator [SEL_5] (rows=242/242 width=91)
+                  Output:["_col0"]
+                  Filter Operator [FIL_23] (rows=242/242 width=91)
+                    predicate:value is not null
+                    TableScan [TS_3] (rows=242/242 width=91)
+                      default@tab_n6,s2,Tbl:COMPLETE,Col:COMPLETE,Output:["value"]
 
 PREHOOK: query: select s1.key as key, s1.value as value from tab_n6 s1 join tab2_n3 s3 on s1.key=s3.key
 PREHOOK: type: QUERY
@@ -728,7 +715,7 @@ Stage-0
     Stage-1
       Map 1 llap
       File Output Operator [FS_10]
-        Merge Join Operator [MERGEJOIN_15] (rows=382/480 width=95)
+        Merge Join Operator [MERGEJOIN_15] (rows=401/480 width=95)
           Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0","_col1"]
         <-Select Operator [SEL_5] (rows=242/242 width=4)
             Output:["_col0"]
@@ -767,41 +754,48 @@ Plan optimized by CBO.
 
 Vertex dependency in root stage
 Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 2 llap
+      Reducer 3 llap
       File Output Operator [FS_16]
-        Merge Join Operator [MERGEJOIN_27] (rows=604/1166 width=95)
-          Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col0","_col1"]
-        <-Map 1 [SIMPLE_EDGE] llap
-          SHUFFLE [RS_12]
-            PartitionCols:_col1
-            Merge Join Operator [MERGEJOIN_25] (rows=382/480 width=95)
-              Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0","_col1"]
-            <-Select Operator [SEL_5] (rows=242/242 width=4)
-                Output:["_col0"]
-                Filter Operator [FIL_23] (rows=242/242 width=4)
-                  predicate:key is not null
-                  TableScan [TS_3] (rows=242/242 width=4)
-                    default@tab2_n3,s3,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
-            <-Select Operator [SEL_2] (rows=242/242 width=95)
-                Output:["_col0","_col1"]
-                Filter Operator [FIL_22] (rows=242/242 width=95)
-                  predicate:(key is not null and value is not null)
-                  TableScan [TS_0] (rows=242/242 width=95)
-                    default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-        <-Map 4 [SIMPLE_EDGE] llap
+        Merge Join Operator [MERGEJOIN_26] (rows=633/1166 width=95)
+          Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col0","_col1"]
+        <-Map 5 [SIMPLE_EDGE] llap
           SHUFFLE [RS_13]
             PartitionCols:_col0
-            Select Operator [SEL_8] (rows=242/242 width=91)
+            Select Operator [SEL_8] (rows=242/242 width=4)
               Output:["_col0"]
-              Filter Operator [FIL_24] (rows=242/242 width=91)
-                predicate:value is not null
-                TableScan [TS_6] (rows=242/242 width=91)
-                  default@tab2_n3,s2,Tbl:COMPLETE,Col:COMPLETE,Output:["value"]
+              Filter Operator [FIL_24] (rows=242/242 width=4)
+                predicate:key is not null
+                TableScan [TS_6] (rows=242/242 width=4)
+                  default@tab2_n3,s3,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
+        <-Reducer 2 [SIMPLE_EDGE] llap
+          SHUFFLE [RS_12]
+            PartitionCols:_col0
+            Merge Join Operator [MERGEJOIN_25] (rows=382/480 width=95)
+              Conds:RS_9._col1=RS_10._col0(Inner),Output:["_col0","_col1"]
+            <-Map 1 [SIMPLE_EDGE] llap
+              SHUFFLE [RS_9]
+                PartitionCols:_col1
+                Select Operator [SEL_2] (rows=242/242 width=95)
+                  Output:["_col0","_col1"]
+                  Filter Operator [FIL_22] (rows=242/242 width=95)
+                    predicate:(key is not null and value is not null)
+                    TableScan [TS_0] (rows=242/242 width=95)
+                      default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+            <-Map 4 [SIMPLE_EDGE] llap
+              SHUFFLE [RS_10]
+                PartitionCols:_col0
+                Select Operator [SEL_5] (rows=242/242 width=91)
+                  Output:["_col0"]
+                  Filter Operator [FIL_23] (rows=242/242 width=91)
+                    predicate:value is not null
+                    TableScan [TS_3] (rows=242/242 width=91)
+                      default@tab2_n3,s2,Tbl:COMPLETE,Col:COMPLETE,Output:["value"]
 
 PREHOOK: query: select count(*) from (select s1.key as key, s1.value as value from tab_n6 s1 join tab_n6 s3 on s1.key=s3.key
 UNION  ALL
@@ -853,7 +847,7 @@ Stage-0
           Output:["_col0"],aggregations:["count()"]
         <-Reducer 3 [CUSTOM_SIMPLE_EDGE] llap
           PARTITION_ONLY_SHUFFLE [RS_22]
-            Merge Join Operator [MERGEJOIN_38] (rows=1009/1646 width=8)
+            Merge Join Operator [MERGEJOIN_38] (rows=1061/1646 width=8)
               Conds:Union 2._col0=RS_19._col0(Inner)
             <-Map 7 [SIMPLE_EDGE] llap
               SHUFFLE [RS_19]
@@ -868,7 +862,7 @@ Stage-0
               <-Map 1 [CONTAINS] llap
                 Reduce Output Operator [RS_18]
                   PartitionCols:_col0
-                  Merge Join Operator [MERGEJOIN_36] (rows=382/480 width=4)
+                  Merge Join Operator [MERGEJOIN_36] (rows=401/480 width=4)
                     Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0"]
                   <-Select Operator [SEL_5] (rows=242/242 width=4)
                       Output:["_col0"]
@@ -927,24 +921,25 @@ POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 8 <- Union 3 (CONTAINS)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE), Union 3 (CONTAINS)
-Reducer 4 <- Map 9 (SIMPLE_EDGE), Union 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
+Map 9 <- Union 4 (CONTAINS)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 5 <- Map 10 (SIMPLE_EDGE), Union 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 5 llap
+      Reducer 6 llap
       File Output Operator [FS_31]
         Group By Operator [GBY_29] (rows=1/1 width=8)
           Output:["_col0"],aggregations:["count()"]
-        <-Reducer 4 [CUSTOM_SIMPLE_EDGE] llap
+        <-Reducer 5 [CUSTOM_SIMPLE_EDGE] llap
           PARTITION_ONLY_SHUFFLE [RS_28]
-            Merge Join Operator [MERGEJOIN_50] (rows=1368/3768 width=8)
-              Conds:Union 3._col0=RS_25._col0(Inner)
-            <-Map 9 [SIMPLE_EDGE] llap
+            Merge Join Operator [MERGEJOIN_49] (rows=1443/3768 width=8)
+              Conds:Union 4._col0=RS_25._col0(Inner)
+            <-Map 10 [SIMPLE_EDGE] llap
               SHUFFLE [RS_25]
                 PartitionCols:_col0
                 Select Operator [SEL_23] (rows=500/500 width=4)
@@ -953,8 +948,8 @@ Stage-0
                     predicate:key is not null
                     TableScan [TS_21] (rows=500/500 width=4)
                       default@tab_part_n7,b_n10,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
-            <-Union 3 [SIMPLE_EDGE]
-              <-Map 8 [CONTAINS] llap
+            <-Union 4 [SIMPLE_EDGE]
+              <-Map 9 [CONTAINS] llap
                 Reduce Output Operator [RS_24]
                   PartitionCols:_col0
                   Select Operator [SEL_18] (rows=242/242 width=4)
@@ -963,37 +958,43 @@ Stage-0
                       predicate:key is not null
                       TableScan [TS_16] (rows=242/242 width=4)
                         Output:["key"]
-              <-Reducer 2 [CONTAINS] llap
+              <-Reducer 3 [CONTAINS] llap
                 Reduce Output Operator [RS_24]
                   PartitionCols:_col0
-                  Merge Join Operator [MERGEJOIN_49] (rows=604/1166 width=4)
-                    Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col0"]
-                  <-Map 1 [SIMPLE_EDGE] llap
-                    SHUFFLE [RS_12]
-                      PartitionCols:_col1
-                      Merge Join Operator [MERGEJOIN_47] (rows=382/480 width=95)
-                        Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0","_col1"]
-                      <-Select Operator [SEL_5] (rows=242/242 width=4)
-                          Output:["_col0"]
-                          Filter Operator [FIL_43] (rows=242/242 width=4)
-                            predicate:key is not null
-                            TableScan [TS_3] (rows=242/242 width=4)
-                              default@tab_n6,s3,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
-                      <-Select Operator [SEL_2] (rows=242/242 width=95)
-                          Output:["_col0","_col1"]
-                          Filter Operator [FIL_42] (rows=242/242 width=95)
-                            predicate:(key is not null and value is not null)
-                            TableScan [TS_0] (rows=242/242 width=95)
-                              default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                  <-Map 7 [SIMPLE_EDGE] llap
+                  Merge Join Operator [MERGEJOIN_48] (rows=633/1166 width=4)
+                    Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col0"]
+                  <-Map 8 [SIMPLE_EDGE] llap
                     SHUFFLE [RS_13]
                       PartitionCols:_col0
-                      Select Operator [SEL_8] (rows=242/242 width=91)
+                      Select Operator [SEL_8] (rows=242/242 width=4)
                         Output:["_col0"]
-                        Filter Operator [FIL_44] (rows=242/242 width=91)
-                          predicate:value is not null
-                          TableScan [TS_6] (rows=242/242 width=91)
-                            default@tab_n6,s2,Tbl:COMPLETE,Col:COMPLETE,Output:["value"]
+                        Filter Operator [FIL_44] (rows=242/242 width=4)
+                          predicate:key is not null
+                          TableScan [TS_6] (rows=242/242 width=4)
+                            default@tab_n6,s3,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
+                  <-Reducer 2 [SIMPLE_EDGE] llap
+                    SHUFFLE [RS_12]
+                      PartitionCols:_col0
+                      Merge Join Operator [MERGEJOIN_47] (rows=382/480 width=4)
+                        Conds:RS_9._col1=RS_10._col0(Inner),Output:["_col0"]
+                      <-Map 1 [SIMPLE_EDGE] llap
+                        SHUFFLE [RS_9]
+                          PartitionCols:_col1
+                          Select Operator [SEL_2] (rows=242/242 width=95)
+                            Output:["_col0","_col1"]
+                            Filter Operator [FIL_42] (rows=242/242 width=95)
+                              predicate:(key is not null and value is not null)
+                              TableScan [TS_0] (rows=242/242 width=95)
+                                default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                      <-Map 7 [SIMPLE_EDGE] llap
+                        SHUFFLE [RS_10]
+                          PartitionCols:_col0
+                          Select Operator [SEL_5] (rows=242/242 width=91)
+                            Output:["_col0"]
+                            Filter Operator [FIL_43] (rows=242/242 width=91)
+                              predicate:value is not null
+                              TableScan [TS_3] (rows=242/242 width=91)
+                                default@tab_n6,s2,Tbl:COMPLETE,Col:COMPLETE,Output:["value"]
 
 PREHOOK: query: CREATE TABLE a_n14(key STRING, value STRING) STORED AS TEXTFILE
 PREHOOK: type: CREATETABLE
@@ -1106,24 +1107,24 @@ POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Union 2 (CONTAINS)
-Map 16 <- Union 17 (CONTAINS)
-Map 18 <- Union 17 (CONTAINS)
-Map 19 <- Union 17 (CONTAINS)
+Map 13 <- Union 14 (CONTAINS)
+Map 15 <- Union 14 (CONTAINS)
+Map 17 <- Union 18 (CONTAINS)
+Map 19 <- Union 18 (CONTAINS)
+Map 20 <- Union 18 (CONTAINS)
 Map 22 <- Union 23 (CONTAINS)
 Map 24 <- Union 23 (CONTAINS)
 Map 25 <- Union 23 (CONTAINS)
 Map 26 <- Union 23 (CONTAINS)
-Map 9 <- Union 2 (CONTAINS)
 Reducer 11 <- Map 10 (SIMPLE_EDGE), Map 21 (SIMPLE_EDGE)
-Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Union 23 (SIMPLE_EDGE), Union 5 (CONTAINS)
-Reducer 14 <- Map 13 (SIMPLE_EDGE), Union 17 (SIMPLE_EDGE)
-Reducer 15 <- Map 20 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE), Union 5 (CONTAINS)
-Reducer 3 <- Map 10 (SIMPLE_EDGE), Union 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS)
-Reducer 6 <- Union 5 (CUSTOM_SIMPLE_EDGE)
-Reducer 7 <- Union 5 (CUSTOM_SIMPLE_EDGE)
-Reducer 8 <- Union 5 (CUSTOM_SIMPLE_EDGE)
+Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Union 23 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Union 14 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 5 <- Union 4 (CUSTOM_SIMPLE_EDGE)
+Reducer 6 <- Union 4 (CUSTOM_SIMPLE_EDGE)
+Reducer 7 <- Union 4 (CUSTOM_SIMPLE_EDGE)
+Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 16 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 8 (SIMPLE_EDGE), Union 18 (SIMPLE_EDGE), Union 4 (CONTAINS)
 
 Stage-5
   Stats Work{}
@@ -1133,38 +1134,38 @@ Stage-5
         Stage-4
           Dependency Collection{}
             Stage-3
-              Reducer 6 llap
+              Reducer 5 llap
               File Output Operator [FS_5]
                 Group By Operator [GBY_3] (rows=1/1 width=880)
                   Output:["_col0","_col1"],aggregations:["compute_stats(VALUE._col0, 'hll')","compute_stats(VALUE._col2, 'hll')"]
-                <-Union 5 [CUSTOM_SIMPLE_EDGE]
+                <-Union 4 [CUSTOM_SIMPLE_EDGE]
                   <-Reducer 12 [CONTAINS] llap
                     File Output Operator [FS_75]
                       table:{"name:":"default.a_n14"}
-                      Select Operator [SEL_72] (rows=192/820 width=175)
+                      Select Operator [SEL_72] (rows=193/820 width=175)
                         Output:["_col0","_col1"]
-                        Merge Join Operator [MERGEJOIN_124] (rows=192/820 width=175)
+                        Merge Join Operator [MERGEJOIN_121] (rows=193/820 width=175)
                           Conds:RS_69._col1=Union 23._col0(Inner),Output:["_col0","_col3"]
                         <-Reducer 11 [SIMPLE_EDGE] llap
                           SHUFFLE [RS_69]
                             PartitionCols:_col1
-                            Merge Join Operator [MERGEJOIN_123] (rows=39/115 width=264)
+                            Merge Join Operator [MERGEJOIN_118] (rows=39/115 width=264)
                               Conds:RS_66._col0=RS_67._col0(Inner),Output:["_col0","_col1","_col3"]
                             <-Map 10 [SIMPLE_EDGE] llap
                               SHUFFLE [RS_66]
                                 PartitionCols:_col0
-                                Select Operator [SEL_10] (rows=25/25 width=175)
+                                Select Operator [SEL_5] (rows=25/25 width=175)
                                   Output:["_col0","_col1"]
-                                  Filter Operator [FIL_106] (rows=25/25 width=175)
+                                  Filter Operator [FIL_102] (rows=25/25 width=175)
                                     predicate:(key is not null and value is not null)
-                                    TableScan [TS_8] (rows=25/25 width=175)
+                                    TableScan [TS_3] (rows=25/25 width=175)
                                       default@src1,x,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
                             <-Map 21 [SIMPLE_EDGE] llap
                               SHUFFLE [RS_67]
                                 PartitionCols:_col0
                                 Select Operator [SEL_51] (rows=25/25 width=175)
                                   Output:["_col0","_col1"]
-                                  Filter Operator [FIL_114] (rows=25/25 width=175)
+                                  Filter Operator [FIL_111] (rows=25/25 width=175)
                                     predicate:key is not null
                                     TableScan [TS_49] (rows=25/25 width=175)
                                       default@src1,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
@@ -1174,7 +1175,7 @@ Stage-5
                               PartitionCols:_col0
                               Select Operator [SEL_54] (rows=25/25 width=89)
                                 Output:["_col0"]
-                                Filter Operator [FIL_115] (rows=25/25 width=89)
+                                Filter Operator [FIL_112] (rows=25/25 width=89)
                                   predicate:value is not null
                                   TableScan [TS_52] (rows=25/25 width=89)
                                     Output:["value"]
@@ -1183,7 +1184,7 @@ Stage-5
                               PartitionCols:_col0
                               Select Operator [SEL_57] (rows=500/500 width=91)
                                 Output:["_col0"]
-                                Filter Operator [FIL_116] (rows=500/500 width=91)
+                                Filter Operator [FIL_113] (rows=500/500 width=91)
                                   predicate:value is not null
                                   TableScan [TS_55] (rows=500/500 width=91)
                                     Output:["value"]
@@ -1192,7 +1193,7 @@ Stage-5
                               PartitionCols:_col0
                               Select Operator [SEL_61] (rows=500/500 width=91)
                                 Output:["_col0"]
-                                Filter Operator [FIL_117] (rows=500/500 width=91)
+                                Filter Operator [FIL_114] (rows=500/500 width=91)
                                   predicate:value is not null
                                   TableScan [TS_59] (rows=500/500 width=91)
                                     Output:["value"]
@@ -1201,175 +1202,175 @@ Stage-5
                               PartitionCols:_col0
                               Select Operator [SEL_64] (rows=500/500 width=91)
                                 Output:["_col0"]
-                                Filter Operator [FIL_118] (rows=500/500 width=91)
+                                Filter Operator [FIL_115] (rows=500/500 width=91)
                                   predicate:value is not null
                                   TableScan [TS_62] (rows=500/500 width=91)
                                     Output:["value"]
                     Reduce Output Operator [RS_2]
-                      Select Operator [SEL_1] (rows=2941/6411 width=178)
+                      Select Operator [SEL_1] (rows=2899/6411 width=178)
                         Output:["key","value"]
                          Please refer to the previous Select Operator [SEL_72]
                     File Output Operator [FS_77]
                       table:{"name:":"default.b_n10"}
                        Please refer to the previous Select Operator [SEL_72]
                     Reduce Output Operator [RS_2]
-                      Select Operator [SEL_1] (rows=2941/6411 width=178)
+                      Select Operator [SEL_1] (rows=2899/6411 width=178)
                         Output:["key","value"]
                          Please refer to the previous Select Operator [SEL_72]
                     File Output Operator [FS_79]
                       table:{"name:":"default.c_n3"}
                        Please refer to the previous Select Operator [SEL_72]
                     Reduce Output Operator [RS_2]
-                      Select Operator [SEL_1] (rows=2941/6411 width=178)
+                      Select Operator [SEL_1] (rows=2899/6411 width=178)
                         Output:["key","value"]
                          Please refer to the previous Select Operator [SEL_72]
-                  <-Reducer 15 [CONTAINS] llap
+                  <-Reducer 3 [CONTAINS] llap
                     File Output Operator [FS_75]
                       table:{"name:":"default.a_n14"}
-                      Select Operator [SEL_44] (rows=2682/5421 width=178)
+                      Select Operator [SEL_20] (rows=66/170 width=177)
                         Output:["_col0","_col1"]
-                        Merge Join Operator [MERGEJOIN_122] (rows=2682/5421 width=178)
-                          Conds:RS_41._col1=RS_42._col0(Inner),Output:["_col1","_col4"]
-                        <-Map 20 [SIMPLE_EDGE] llap
-                          SHUFFLE [RS_42]
-                            PartitionCols:_col0
-                            Select Operator [SEL_37] (rows=500/500 width=178)
-                              Output:["_col0","_col1"]
-                              Filter Operator [FIL_112] (rows=500/500 width=178)
-                                predicate:key is not null
-                                TableScan [TS_35] (rows=500/500 width=178)
-                                  default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                        <-Reducer 14 [SIMPLE_EDGE] llap
-                          SHUFFLE [RS_41]
-                            PartitionCols:_col1
-                            Merge Join Operator [MERGEJOIN_121] (rows=1658/2097 width=87)
-                              Conds:Union 17._col0=RS_39._col1(Inner),Output:["_col1"]
-                            <-Map 13 [SIMPLE_EDGE] llap
-                              SHUFFLE [RS_39]
-                                PartitionCols:_col1
-                                Select Operator [SEL_34] (rows=500/500 width=178)
+                        Merge Join Operator [MERGEJOIN_119] (rows=66/170 width=177)
+                          Conds:RS_17._col3=Union 14._col0(Inner),Output:["_col1","_col2"]
+                        <-Reducer 2 [SIMPLE_EDGE] llap
+                          SHUFFLE [RS_17]
+                            PartitionCols:_col3
+                            Merge Join Operator [MERGEJOIN_116] (rows=39/37 width=266)
+                              Conds:RS_14._col0=RS_15._col0(Inner),Output:["_col1","_col2","_col3"]
+                            <-Map 1 [SIMPLE_EDGE] llap
+                              SHUFFLE [RS_14]
+                                PartitionCols:_col0
+                                Select Operator [SEL_2] (rows=500/500 width=178)
                                   Output:["_col0","_col1"]
-                                  Filter Operator [FIL_111] (rows=500/500 width=178)
-                                    predicate:(key is not null and value is not null)
-                                    TableScan [TS_11] (rows=500/500 width=178)
+                                  Filter Operator [FIL_101] (rows=500/500 width=178)
+                                    predicate:key is not null
+                                    TableScan [TS_0] (rows=500/500 width=178)
                                       default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                            <-Union 17 [SIMPLE_EDGE]
-                              <-Map 16 [CONTAINS] llap
-                                Reduce Output Operator [RS_38]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_23] (rows=25/25 width=89)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_108] (rows=25/25 width=89)
-                                      predicate:value is not null
-                                      TableScan [TS_21] (rows=25/25 width=89)
-                                        Output:["value"]
-                              <-Map 18 [CONTAINS] llap
-                                Reduce Output Operator [RS_38]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_26] (rows=500/500 width=91)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_109] (rows=500/500 width=91)
-                                      predicate:value is not null
-                                      TableScan [TS_24] (rows=500/500 width=91)
-                                        Output:["value"]
-                              <-Map 19 [CONTAINS] llap
-                                Reduce Output Operator [RS_38]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_30] (rows=500/500 width=91)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_110] (rows=500/500 width=91)
-                                      predicate:value is not null
-                                      TableScan [TS_28] (rows=500/500 width=91)
-                                        Output:["value"]
+                            <-Map 10 [SIMPLE_EDGE] llap
+                              SHUFFLE [RS_15]
+                                PartitionCols:_col0
+                                 Please refer to the previous Select Operator [SEL_5]
+                        <-Union 14 [SIMPLE_EDGE]
+                          <-Map 13 [CONTAINS] llap
+                            Reduce Output Operator [RS_18]
+                              PartitionCols:_col0
+                              Select Operator [SEL_8] (rows=25/25 width=89)
+                                Output:["_col0"]
+                                Filter Operator [FIL_103] (rows=25/25 width=89)
+                                  predicate:value is not null
+                                  TableScan [TS_6] (rows=25/25 width=89)
+                                    Output:["value"]
+                          <-Map 15 [CONTAINS] llap
+                            Reduce Output Operator [RS_18]
+                              PartitionCols:_col0
+                              Select Operator [SEL_11] (rows=500/500 width=91)
+                                Output:["_col0"]
+                                Filter Operator [FIL_104] (rows=500/500 width=91)
+                                  predicate:value is not null
+                                  TableScan [TS_9] (rows=500/500 width=91)
+                                    Output:["value"]
                     Reduce Output Operator [RS_2]
-                      Select Operator [SEL_1] (rows=2941/6411 width=178)
+                      Select Operator [SEL_1] (rows=2899/6411 width=178)
                         Output:["key","value"]
-                         Please refer to the previous Select Operator [SEL_44]
+                         Please refer to the previous Select Operator [SEL_20]
                     File Output Operator [FS_77]
                       table:{"name:":"default.b_n10"}
-                       Please refer to the previous Select Operator [SEL_44]
+                       Please refer to the previous Select Operator [SEL_20]
                     Reduce Output Operator [RS_2]
-                      Select Operator [SEL_1] (rows=2941/6411 width=178)
+                      Select Operator [SEL_1] (rows=2899/6411 width=178)
                         Output:["key","value"]
-                         Please refer to the previous Select Operator [SEL_44]
+                         Please refer to the previous Select Operator [SEL_20]
                     File Output Operator [FS_79]
                       table:{"name:":"default.c_n3"}
-                       Please refer to the previous Select Operator [SEL_44]
+                       Please refer to the previous Select Operator [SEL_20]
                     Reduce Output Operator [RS_2]
-                      Select Operator [SEL_1] (rows=2941/6411 width=178)
+                      Select Operator [SEL_1] (rows=2899/6411 width=178)
                         Output:["key","value"]
-                         Please refer to the previous Select Operator [SEL_44]
-                  <-Reducer 4 [CONTAINS] llap
+                         Please refer to the previous Select Operator [SEL_20]
+                  <-Reducer 9 [CONTAINS] llap
                     File Output Operator [FS_75]
                       table:{"name:":"default.a_n14"}
-                      Select Operator [SEL_20] (rows=67/170 width=177)
+                      Select Operator [SEL_44] (rows=2640/5421 width=178)
                         Output:["_col0","_col1"]
-                        Merge Join Operator [MERGEJOIN_120] (rows=67/170 width=177)
-                          Conds:RS_17._col1=RS_18._col0(Inner),Output:["_col1","_col4"]
-                        <-Map 13 [SIMPLE_EDGE] llap
-                          SHUFFLE [RS_18]
-                            PartitionCols:_col0
-                            Select Operator [SEL_13] (rows=500/500 width=178)
-                              Output:["_col0","_col1"]
-                              Filter Operator [FIL_107] (rows=500/500 width=178)
-                                predicate:key is not null
-                                 Please refer to the previous TableScan [TS_11]
-                        <-Reducer 3 [SIMPLE_EDGE] llap
-                          SHUFFLE [RS_17]
+                        Merge Join Operator [MERGEJOIN_120] (rows=2640/5421 width=178)
+                          Conds:RS_41._col1=Union 18._col0(Inner),Output:["_col0","_col3"]
+                        <-Reducer 8 [SIMPLE_EDGE] llap
+                          SHUFFLE [RS_41]
                             PartitionCols:_col1
-                            Merge Join Operator [MERGEJOIN_119] (rows=42/108 width=86)
-                              Conds:Union 2._col0=RS_15._col1(Inner),Output:["_col1"]
-                            <-Map 10 [SIMPLE_EDGE] llap
-                              SHUFFLE [RS_15]
-                                PartitionCols:_col1
-                                 Please refer to the previous Select Operator [SEL_10]
-                            <-Union 2 [SIMPLE_EDGE]
-                              <-Map 1 [CONTAINS] llap
-                                Reduce Output Operator [RS_14]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_2] (rows=25/25 width=89)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_104] (rows=25/25 width=89)
-                                      predicate:value is not null
-                                      TableScan [TS_0] (rows=25/25 width=89)
-                                        Output:["value"]
-                              <-Map 9 [CONTAINS] llap
-                                Reduce Output Operator [RS_14]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_5] (rows=500/500 width=91)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_105] (rows=500/500 width=91)
-                                      predicate:value is not null
-                                      TableScan [TS_3] (rows=500/500 width=91)
-                                        Output:["value"]
+                            Merge Join Operator [MERGEJOIN_117] (rows=791/1028 width=269)
+                              Conds:RS_38._col0=RS_39._col0(Inner),Output:["_col0","_col1","_col3"]
+                            <-Map 1 [SIMPLE_EDGE] llap
+                              SHUFFLE [RS_38]
+                                PartitionCols:_col0
+                                Select Operator [SEL_23] (rows=500/500 width=178)
+                                  Output:["_col0","_col1"]
+                                  Filter Operator [FIL_105] (rows=500/500 width=178)
+                                    predicate:(key is not null and value is not null)
+                                     Please refer to the previous TableScan [TS_0]
+                            <-Map 16 [SIMPLE_EDGE] llap
+                              SHUFFLE [RS_39]
+                                PartitionCols:_col0
+                                Select Operator [SEL_26] (rows=500/500 width=178)
+                                  Output:["_col0","_col1"]
+                                  Filter Operator [FIL_106] (rows=500/500 width=178)
+                                    predicate:key is not null
+                                    TableScan [TS_24] (rows=500/500 width=178)
+                                      default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                        <-Union 18 [SIMPLE_EDGE]
+                          <-Map 17 [CONTAINS] llap
+                            Reduce Output Operator [RS_42]
+                              PartitionCols:_col0
+                              Select Operator [SEL_29] (rows=25/25 width=89)
+                                Output:["_col0"]
+                                Filter Operator [FIL_107] (rows=25/25 width=89)
+                                  predicate:value is not null
+                                  TableScan [TS_27] (rows=25/25 width=89)
+                                    Output:["value"]
+                          <-Map 19 [CONTAINS] llap
+                            Reduce Output Operator [RS_42]
+                              PartitionCols:_col0
+                              Select Operator [SEL_32] (rows=500/500 width=91)
+                                Output:["_col0"]
+                                Filter Operator [FIL_108] (rows=500/500 width=91)
+                                  predicate:value is not null
+                                  TableScan [TS_30] (rows=500/500 width=91)
+                                    Output:["value"]
+                          <-Map 20 [CONTAINS] llap
+                            Reduce Output Operator [RS_42]
+                              PartitionCols:_col0
+                              Select Operator [SEL_36] (rows=500/500 width=91)
+                                Output:["_col0"]
+                                Filter Operator [FIL_109] (rows=500/500 width=91)
+                                  predicate:value is not null
+                                  TableScan [TS_34] (rows=500/500 width=91)
+                                    Output:["value"]
                     Reduce Output Operator [RS_2]
-                      Select Operator [SEL_1] (rows=2941/6411 width=178)
+                      Select Operator [SEL_1] (rows=2899/6411 width=178)
                         Output:["key","value"]
-                         Please refer to the previous Select Operator [SEL_20]
+                         Please refer to the previous Select Operator [SEL_44]
                     File Output Operator [FS_77]
                       table:{"name:":"default.b_n10"}
-                       Please refer to the previous Select Operator [SEL_20]
+                       Please refer to the previous Select Operator [SEL_44]
                     Reduce Output Operator [RS_2]
-                      Select Operator [SEL_1] (rows=2941/6411 width=178)
+                      Select Operator [SEL_1] (rows=2899/6411 width=178)
                         Output:["key","value"]
-                         Please refer to the previous Select Operator [SEL_20]
+                         Please refer to the previous Select Operator [SEL_44]
                     File Output Operator [FS_79]
                       table:{"name:":"default.c_n3"}
-                       Please refer to the previous Select Operator [SEL_20]
+                       Please refer to the previous Select Operator [SEL_44]
                     Reduce Output Operator [RS_2]
-                      Select Operator [SEL_1] (rows=2941/6411 width=178)
+                      Select Operator [SEL_1] (rows=2899/6411 width=178)
                         Output:["key","value"]
-                         Please refer to the previous Select Operator [SEL_20]
-              Reducer 7 llap
+                         Please refer to the previous Select Operator [SEL_44]
+              Reducer 6 llap
               File Output Operator [FS_5]
                 Group By Operator [GBY_3] (rows=1/1 width=880)
                   Output:["_col0","_col1"],aggregations:["compute_stats(VALUE._col0, 'hll')","compute_stats(VALUE._col2, 'hll')"]
-                <- Please refer to the previous Union 5 [CUSTOM_SIMPLE_EDGE]
-              Reducer 8 llap
+                <- Please refer to the previous Union 4 [CUSTOM_SIMPLE_EDGE]
+              Reducer 7 llap
               File Output Operator [FS_5]
                 Group By Operator [GBY_3] (rows=1/1 width=880)
                   Output:["_col0","_col1"],aggregations:["compute_stats(VALUE._col0, 'hll')","compute_stats(VALUE._col2, 'hll')"]
-                <- Please refer to the previous Union 5 [CUSTOM_SIMPLE_EDGE]
+                <- Please refer to the previous Union 4 [CUSTOM_SIMPLE_EDGE]
 Stage-6
   Stats Work{}
     Stage-1
@@ -1458,32 +1459,32 @@ POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Union 2 (CONTAINS)
-Map 13 <- Union 2 (CONTAINS)
-Map 20 <- Union 21 (CONTAINS)
-Map 25 <- Union 21 (CONTAINS)
-Map 26 <- Union 23 (CONTAINS)
+Map 16 <- Union 17 (CONTAINS)
+Map 19 <- Union 17 (CONTAINS)
+Map 21 <- Union 22 (CONTAINS)
+Map 26 <- Union 22 (CONTAINS)
+Map 27 <- Union 24 (CONTAINS)
 Map 29 <- Union 30 (CONTAINS)
 Map 36 <- Union 30 (CONTAINS)
 Map 37 <- Union 32 (CONTAINS)
 Map 38 <- Union 34 (CONTAINS)
-Reducer 10 <- Reducer 9 (CUSTOM_SIMPLE_EDGE)
-Reducer 11 <- Reducer 9 (CUSTOM_SIMPLE_EDGE)
-Reducer 12 <- Reducer 9 (CUSTOM_SIMPLE_EDGE)
-Reducer 15 <- Map 14 (SIMPLE_EDGE), Map 28 (SIMPLE_EDGE)
-Reducer 16 <- Reducer 15 (SIMPLE_EDGE), Reducer 35 (SIMPLE_EDGE), Union 8 (CONTAINS)
-Reducer 18 <- Map 17 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
-Reducer 19 <- Map 27 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 22 <- Union 21 (SIMPLE_EDGE), Union 23 (CONTAINS)
-Reducer 24 <- Union 23 (SIMPLE_EDGE)
-Reducer 3 <- Union 2 (SIMPLE_EDGE)
+Reducer 10 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
+Reducer 11 <- Map 1 (SIMPLE_EDGE), Map 20 (SIMPLE_EDGE)
+Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 25 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 14 <- Map 13 (SIMPLE_EDGE), Map 28 (SIMPLE_EDGE)
+Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 35 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 18 <- Union 17 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE)
+Reducer 23 <- Union 22 (SIMPLE_EDGE), Union 24 (CONTAINS)
+Reducer 25 <- Union 24 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 18 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
 Reducer 31 <- Union 30 (SIMPLE_EDGE), Union 32 (CONTAINS)
 Reducer 33 <- Union 32 (SIMPLE_EDGE), Union 34 (CONTAINS)
 Reducer 35 <- Union 34 (SIMPLE_EDGE)
-Reducer 4 <- Map 14 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 17 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 7 <- Union 6 (SIMPLE_EDGE), Union 8 (CONTAINS)
-Reducer 9 <- Union 8 (SIMPLE_EDGE)
+Reducer 5 <- Union 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 7 <- Union 6 (SIMPLE_EDGE)
+Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
+Reducer 9 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
 
 Stage-5
   Stats Work{}
@@ -1497,40 +1498,40 @@ Stage-5
               File Output Operator [FS_5]
                 Group By Operator [GBY_3] (rows=1/1 width=880)
                   Output:["_col0","_col1"],aggregations:["compute_stats(VALUE._col0, 'hll')","compute_stats(VALUE._col2, 'hll')"]
-                <-Reducer 9 [CUSTOM_SIMPLE_EDGE] llap
+                <-Reducer 7 [CUSTOM_SIMPLE_EDGE] llap
                   PARTITION_ONLY_SHUFFLE [RS_2]
-                    Select Operator [SEL_1] (rows=2941/319 width=178)
+                    Select Operator [SEL_1] (rows=2899/319 width=178)
                       Output:["key","value"]
-                      Group By Operator [GBY_112] (rows=2941/319 width=178)
+                      Group By Operator [GBY_112] (rows=2899/319 width=178)
                         Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                      <-Union 8 [SIMPLE_EDGE]
-                        <-Reducer 16 [CONTAINS] llap
+                      <-Union 6 [SIMPLE_EDGE]
+                        <-Reducer 15 [CONTAINS] llap
                           Reduce Output Operator [RS_111]
                             PartitionCols:_col0, _col1
-                            Select Operator [SEL_107] (rows=192/304 width=175)
+                            Select Operator [SEL_107] (rows=193/304 width=175)
                               Output:["_col0","_col1"]
-                              Merge Join Operator [MERGEJOIN_164] (rows=192/304 width=175)
+                              Merge Join Operator [MERGEJOIN_161] (rows=193/304 width=175)
                                 Conds:RS_104._col1=RS_105._col1(Inner),Output:["_col0","_col3"]
-                              <-Reducer 15 [SIMPLE_EDGE] llap
+                              <-Reducer 14 [SIMPLE_EDGE] llap
                                 SHUFFLE [RS_104]
                                   PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_163] (rows=39/115 width=264)
+                                  Merge Join Operator [MERGEJOIN_158] (rows=39/115 width=264)
                                     Conds:RS_101._col0=RS_102._col0(Inner),Output:["_col0","_col1","_col3"]
-                                  <-Map 14 [SIMPLE_EDGE] llap
+                                  <-Map 13 [SIMPLE_EDGE] llap
                                     SHUFFLE [RS_101]
                                       PartitionCols:_col0
-                                      Select Operator [SEL_14] (rows=25/25 width=175)
+                                      Select Operator [SEL_5] (rows=25/25 width=175)
                                         Output:["_col0","_col1"]
-                                        Filter Operator [FIL_146] (rows=25/25 width=175)
+         

<TRUNCATED>