You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2018/10/13 18:16:12 UTC

[6/6] hive git commit: HIVE-20704: Extend HivePreFilteringRule to support other functions (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

HIVE-20704: Extend HivePreFilteringRule to support other functions (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: f0b76e2407acc0239a5bace2be5820da8d26c1df
Parents: 5ace1f7
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Sat Oct 13 11:15:58 2018 -0700
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Sat Oct 13 11:15:58 2018 -0700

----------------------------------------------------------------------
 .../calcite/rules/HivePreFilteringRule.java     |  40 +--
 .../druid/druidmini_expressions.q.out           |   2 +-
 .../clientpositive/filter_cond_pushdown.q.out   |  32 +-
 .../clientpositive/perf/spark/query13.q.out     | 220 +++++++------
 .../clientpositive/perf/spark/query47.q.out     | 104 +++---
 .../clientpositive/perf/spark/query48.q.out     | 122 +++----
 .../clientpositive/perf/spark/query53.q.out     |   4 +-
 .../clientpositive/perf/spark/query57.q.out     | 104 +++---
 .../clientpositive/perf/spark/query63.q.out     |   4 +-
 .../clientpositive/perf/spark/query85.q.out     | 328 ++++++++++---------
 .../clientpositive/perf/spark/query88.q.out     | 304 ++++++++---------
 .../clientpositive/perf/spark/query89.q.out     |  92 +++---
 .../clientpositive/perf/tez/query13.q.out       | 280 ++++++++--------
 .../clientpositive/perf/tez/query47.q.out       |  22 +-
 .../clientpositive/perf/tez/query48.q.out       | 220 ++++++-------
 .../clientpositive/perf/tez/query53.q.out       |   2 +-
 .../clientpositive/perf/tez/query57.q.out       |  22 +-
 .../clientpositive/perf/tez/query63.q.out       |   2 +-
 .../clientpositive/perf/tez/query85.q.out       | 278 ++++++++--------
 .../clientpositive/perf/tez/query88.q.out       | 298 ++++++++---------
 .../clientpositive/perf/tez/query89.q.out       |  46 +--
 .../results/clientpositive/pointlookup5.q.out   |   8 +-
 22 files changed, 1250 insertions(+), 1284 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f0b76e24/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
index 5d90c87..33205a5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
 import java.util.ArrayList;
-import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -32,8 +31,6 @@ import org.apache.calcite.rel.core.RelFactories.FilterFactory;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
@@ -51,13 +48,7 @@ public class HivePreFilteringRule extends RelOptRule {
 
   protected static final Logger LOG = LoggerFactory.getLogger(HivePreFilteringRule.class);
 
-  private static final Set<SqlKind>        COMPARISON = EnumSet.of(SqlKind.EQUALS,
-                                                          SqlKind.GREATER_THAN_OR_EQUAL,
-                                                          SqlKind.LESS_THAN_OR_EQUAL,
-                                                          SqlKind.GREATER_THAN, SqlKind.LESS_THAN,
-                                                          SqlKind.NOT_EQUALS);
-
-  private final FilterFactory              filterFactory;
+  private final FilterFactory filterFactory;
 
   // Max number of nodes when converting to CNF
   private final int maxCNFNodeCount;
@@ -120,7 +111,7 @@ public class HivePreFilteringRule extends RelOptRule {
 
       for (RexNode operand : operands) {
         if (operand.getKind() == SqlKind.OR) {
-          extractedCommonOperands = extractCommonOperands(rexBuilder, operand, maxCNFNodeCount);
+          extractedCommonOperands = extractCommonOperands(rexBuilder, filter.getInput(), operand, maxCNFNodeCount);
           for (RexNode extractedExpr : extractedCommonOperands) {
             if (operandsToPushDownDigest.add(extractedExpr.toString())) {
               operandsToPushDown.add(extractedExpr);
@@ -155,7 +146,7 @@ public class HivePreFilteringRule extends RelOptRule {
       break;
 
     case OR:
-      operandsToPushDown = extractCommonOperands(rexBuilder, topFilterCondition, maxCNFNodeCount);
+      operandsToPushDown = extractCommonOperands(rexBuilder, filter.getInput(), topFilterCondition, maxCNFNodeCount);
       break;
     default:
       return;
@@ -191,8 +182,8 @@ public class HivePreFilteringRule extends RelOptRule {
 
   }
 
-  private static List<RexNode> extractCommonOperands(RexBuilder rexBuilder, RexNode condition,
-          int maxCNFNodeCount) {
+  private static List<RexNode> extractCommonOperands(RexBuilder rexBuilder, RelNode input,
+      RexNode condition, int maxCNFNodeCount) {
     assert condition.getKind() == SqlKind.OR;
     Multimap<String, RexNode> reductionCondition = LinkedHashMultimap.create();
 
@@ -216,27 +207,12 @@ public class HivePreFilteringRule extends RelOptRule {
           return new ArrayList<>();
         }
         RexCall conjCall = (RexCall) conjunction;
-        RexNode ref = null;
-        if (COMPARISON.contains(conjCall.getOperator().getKind())) {
-          if (conjCall.operands.get(0) instanceof RexInputRef
-              && conjCall.operands.get(1) instanceof RexLiteral) {
-            ref = conjCall.operands.get(0);
-          } else if (conjCall.operands.get(1) instanceof RexInputRef
-              && conjCall.operands.get(0) instanceof RexLiteral) {
-            ref = conjCall.operands.get(1);
-          } else {
-            // We do not know what it is, we bail out for safety
-            return new ArrayList<>();
-          }
-        } else if (conjCall.getOperator().getKind().equals(SqlKind.IN)) {
-          ref = conjCall.operands.get(0);
-        } else if (conjCall.getOperator().getKind().equals(SqlKind.BETWEEN)) {
-          ref = conjCall.operands.get(1);
-        } else {
+        Set<Integer> refs = HiveCalciteUtil.getInputRefs(conjCall);
+        if (refs.size() != 1) {
           // We do not know what it is, we bail out for safety
           return new ArrayList<>();
         }
-
+        RexNode ref = rexBuilder.makeInputRef(input, refs.iterator().next());
         String stringRef = ref.toString();
         reductionCondition.put(stringRef, conjCall);
         refsInCurrentOperand.add(stringRef);

http://git-wip-us.apache.org/repos/asf/hive/blob/f0b76e24/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out b/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out
index 59285e8..9c9af44 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out
@@ -144,7 +144,7 @@ STAGE PLANS:
           properties:
             druid.fieldNames $f0,_o__c1,_o__c2,_o__c3,$f4,$f5
             druid.fieldTypes double,int,bigint,double,bigint,bigint
-            druid.query.json {"queryType":"timeseries","dataSource":"default.druid_table_alltypesorc","descending":false,"granularity":"all","filter":{"type":"or","fields":[{"type":"and","fields":[{"type":"expression","expression":"(ceil(\"cfloat\") > 0)"},{"type":"expression","expression":"((floor(\"cdouble\") * 2) < 1000)"}]},{"type":"and","fields":[{"type":"expression","expression":"((log(\"cdouble\") / 1.0) > 0)"},{"type":"expression","expression":"(cos(\"cint\") > 0)"}]},{"type":"expression","expression":"(sin(\"cdouble\") > 1)"}]},"aggregations":[{"type":"doubleSum","name":"$f0","expression":"(\"cfloat\" + CAST(1, 'DOUBLE'))"},{"type":"doubleSum","name":"$f1","expression":"(\"cdouble\" + CAST(\"ctinyint\", 'DOUBLE'))"},{"type":"longSum","name":"$f2","fieldName":"ctinyint"},{"type":"longSum","name":"$f3","fieldName":"csmallint"},{"type":"longSum","name":"$f4","fieldName":"cint"},{"type":"longSum","name":"$f5","fieldName":"cbigint"}],"postAggregations":[{"type":"expression","nam
 e":"_o__c1","expression":"CAST(\"$f1\", 'LONG')"},{"type":"expression","name":"_o__c2","expression":"(\"$f2\" + 1)"},{"type":"expression","name":"_o__c3","expression":"CAST((\"$f3\" + \"$f4\"), 'DOUBLE')"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"context":{"skipEmptyBuckets":false}}
+            druid.query.json {"queryType":"timeseries","dataSource":"default.druid_table_alltypesorc","descending":false,"granularity":"all","filter":{"type":"and","fields":[{"type":"or","fields":[{"type":"expression","expression":"((floor(\"cdouble\") * 2) < 1000)"},{"type":"expression","expression":"((log(\"cdouble\") / 1.0) > 0)"},{"type":"expression","expression":"(sin(\"cdouble\") > 1)"}]},{"type":"or","fields":[{"type":"and","fields":[{"type":"expression","expression":"(ceil(\"cfloat\") > 0)"},{"type":"expression","expression":"((floor(\"cdouble\") * 2) < 1000)"}]},{"type":"and","fields":[{"type":"expression","expression":"((log(\"cdouble\") / 1.0) > 0)"},{"type":"expression","expression":"(cos(\"cint\") > 0)"}]},{"type":"expression","expression":"(sin(\"cdouble\") > 1)"}]}]},"aggregations":[{"type":"doubleSum","name":"$f0","expression":"(\"cfloat\" + CAST(1, 'DOUBLE'))"},{"type":"doubleSum","name":"$f1","expression":"(\"cdouble\" + CAST(\"ctinyint\", 'DOUBLE'))"},{"type":"lon
 gSum","name":"$f2","fieldName":"ctinyint"},{"type":"longSum","name":"$f3","fieldName":"csmallint"},{"type":"longSum","name":"$f4","fieldName":"cint"},{"type":"longSum","name":"$f5","fieldName":"cbigint"}],"postAggregations":[{"type":"expression","name":"_o__c1","expression":"CAST(\"$f1\", 'LONG')"},{"type":"expression","name":"_o__c2","expression":"(\"$f2\" + 1)"},{"type":"expression","name":"_o__c3","expression":"CAST((\"$f3\" + \"$f4\"), 'DOUBLE')"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"context":{"skipEmptyBuckets":false}}
             druid.query.type timeseries
           Select Operator
             expressions: $f0 (type: double), _o__c1 (type: int), _o__c2 (type: bigint), _o__c3 (type: double), $f4 (type: bigint), $f5 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/f0b76e24/ql/src/test/results/clientpositive/filter_cond_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/filter_cond_pushdown.q.out b/ql/src/test/results/clientpositive/filter_cond_pushdown.q.out
index c3275aa..5209b44 100644
--- a/ql/src/test/results/clientpositive/filter_cond_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/filter_cond_pushdown.q.out
@@ -151,11 +151,11 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: f
-            filterExpr: key is not null (type: boolean)
+            alias: m
+            filterExpr: ((value <> '') and key is not null) (type: boolean)
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
+              predicate: ((value <> '') and key 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)
@@ -168,11 +168,11 @@ STAGE PLANS:
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: string)
           TableScan
-            alias: m
-            filterExpr: ((value <> '') and key is not null) (type: boolean)
+            alias: f
+            filterExpr: ((value) IN ('2008-04-08', '2008-04-10', '2008-04-09') and key is not null) (type: boolean)
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: ((value <> '') and key is not null) (type: boolean)
+              predicate: ((value) IN ('2008-04-08', '2008-04-10', '2008-04-09') and key 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)
@@ -191,10 +191,10 @@ STAGE PLANS:
           keys:
             0 _col0 (type: string)
             1 _col0 (type: string)
-          outputColumnNames: _col0, _col1, _col3
+          outputColumnNames: _col1, _col2, _col3
           Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
           Filter Operator
-            predicate: (((_col1) IN ('2008-04-08', '2008-04-10') and (_col3 = '2008-04-08')) or (_col1 = '2008-04-09')) (type: boolean)
+            predicate: (((_col3) IN ('2008-04-08', '2008-04-10') and (_col1 = '2008-04-08')) or (_col3 = '2008-04-09')) (type: boolean)
             Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
@@ -208,11 +208,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col3 (type: string)
+              key expressions: _col1 (type: string)
               sort order: +
-              Map-reduce partition columns: _col3 (type: string)
+              Map-reduce partition columns: _col1 (type: string)
               Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-              value expressions: _col0 (type: string)
+              value expressions: _col2 (type: string)
           TableScan
             alias: g
             filterExpr: (value <> '') (type: boolean)
@@ -234,12 +234,12 @@ STAGE PLANS:
           condition map:
                Inner Join 0 to 1
           keys:
-            0 _col3 (type: string)
+            0 _col1 (type: string)
             1 _col0 (type: string)
-          outputColumnNames: _col0, _col4
+          outputColumnNames: _col2, _col4
           Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: string), _col4 (type: string)
+            expressions: _col2 (type: string), _col4 (type: string)
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
@@ -443,10 +443,10 @@ STAGE PLANS:
                   value expressions: _col1 (type: string)
           TableScan
             alias: m
-            filterExpr: ((value <> '') and key is not null) (type: boolean)
+            filterExpr: ((value) IN ('2008-04-10', '2008-04-08') and (value <> '') and key is not null) (type: boolean)
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: ((value <> '') and key is not null) (type: boolean)
+              predicate: ((value <> '') and (value) IN ('2008-04-10', '2008-04-08') and key 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)

http://git-wip-us.apache.org/repos/asf/hive/blob/f0b76e24/ql/src/test/results/clientpositive/perf/spark/query13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/spark/query13.q.out b/ql/src/test/results/clientpositive/perf/spark/query13.q.out
index 6e03bbc..4d530fc 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query13.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query13.q.out
@@ -123,22 +123,22 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 10 
+        Map 8 
             Map Operator Tree:
                 TableScan
-                  alias: store
-                  filterExpr: s_store_sk is not null (type: boolean)
-                  Statistics: Num rows: 1704 Data size: 3256276 Basic stats: COMPLETE Column stats: NONE
+                  alias: household_demographics
+                  filterExpr: ((hd_dep_count) IN (3, 1) and hd_demo_sk is not null) (type: boolean)
+                  Statistics: Num rows: 7200 Data size: 770400 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: s_store_sk is not null (type: boolean)
-                    Statistics: Num rows: 1704 Data size: 3256276 Basic stats: COMPLETE Column stats: NONE
+                    predicate: ((hd_dep_count) IN (3, 1) and hd_demo_sk is not null) (type: boolean)
+                    Statistics: Num rows: 7200 Data size: 770400 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: s_store_sk (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 1704 Data size: 3256276 Basic stats: COMPLETE Column stats: NONE
+                      expressions: hd_demo_sk (type: int), hd_dep_count (type: int)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 7200 Data size: 770400 Basic stats: COMPLETE Column stats: NONE
                       Spark HashTable Sink Operator
                         keys:
-                          0 _col4 (type: int)
+                          0 _col3 (type: int)
                           1 _col0 (type: int)
             Execution mode: vectorized
             Local Work:
@@ -148,23 +148,23 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 8 
+        Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: household_demographics
-                  filterExpr: ((hd_dep_count) IN (3, 1) and hd_demo_sk is not null) (type: boolean)
-                  Statistics: Num rows: 7200 Data size: 770400 Basic stats: COMPLETE Column stats: NONE
+                  alias: store
+                  filterExpr: s_store_sk is not null (type: boolean)
+                  Statistics: Num rows: 1704 Data size: 3256276 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: ((hd_dep_count) IN (3, 1) and hd_demo_sk is not null) (type: boolean)
-                    Statistics: Num rows: 7200 Data size: 770400 Basic stats: COMPLETE Column stats: NONE
+                    predicate: s_store_sk is not null (type: boolean)
+                    Statistics: Num rows: 1704 Data size: 3256276 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: hd_demo_sk (type: int), hd_dep_count (type: int)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 7200 Data size: 770400 Basic stats: COMPLETE Column stats: NONE
+                      expressions: s_store_sk (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1704 Data size: 3256276 Basic stats: COMPLETE Column stats: NONE
                       Spark HashTable Sink Operator
                         keys:
-                          0 _col2 (type: int)
-                          1 _col0 (type: int)
+                          0 _col0 (type: int)
+                          1 _col4 (type: int)
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
@@ -172,33 +172,65 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 133), Map 6 (PARTITION-LEVEL SORT, 133)
-        Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 152), Reducer 2 (PARTITION-LEVEL SORT, 152)
-        Reducer 4 <- Map 9 (PARTITION-LEVEL SORT, 166), Reducer 3 (PARTITION-LEVEL SORT, 166)
-        Reducer 5 <- Reducer 4 (GROUP, 1)
+        Reducer 3 <- Map 2 (PARTITION-LEVEL SORT, 49), Map 7 (PARTITION-LEVEL SORT, 49)
+        Reducer 4 <- Map 9 (PARTITION-LEVEL SORT, 218), Reducer 3 (PARTITION-LEVEL SORT, 218)
+        Reducer 5 <- Map 10 (PARTITION-LEVEL SORT, 28), Reducer 4 (PARTITION-LEVEL SORT, 28)
+        Reducer 6 <- Reducer 5 (GROUP, 1)
 #### A masked pattern was here ####
       Vertices:
-        Map 1 
+        Map 10 
             Map Operator Tree:
                 TableScan
-                  alias: store_sales
-                  filterExpr: ((ss_sales_price BETWEEN 100 AND 150 or ss_sales_price BETWEEN 50 AND 100 or ss_sales_price BETWEEN 150 AND 200) and ss_store_sk is not null and ss_cdemo_sk is not null and ss_hdemo_sk is not null and ss_addr_sk is not null and ss_sold_date_sk is not null) (type: boolean)
-                  Statistics: Num rows: 575995635 Data size: 50814502088 Basic stats: COMPLETE Column stats: NONE
+                  alias: customer_demographics
+                  filterExpr: ((cd_marital_status) IN ('M', 'D', 'U') and (cd_education_status) IN ('4 yr Degree', 'Primary', 'Advanced Degree') and cd_demo_sk is not null) (type: boolean)
+                  Statistics: Num rows: 1861800 Data size: 717186159 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: ((ss_sales_price BETWEEN 100 AND 150 or ss_sales_price BETWEEN 50 AND 100 or ss_sales_price BETWEEN 150 AND 200) and ss_addr_sk is not null and ss_cdemo_sk is not null and ss_hdemo_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null) (type: boolean)
-                    Statistics: Num rows: 191998545 Data size: 16938167362 Basic stats: COMPLETE Column stats: NONE
+                    predicate: ((cd_education_status) IN ('4 yr Degree', 'Primary', 'Advanced Degree') and (cd_marital_status) IN ('M', 'D', 'U') and cd_demo_sk is not null) (type: boolean)
+                    Statistics: Num rows: 1861800 Data size: 717186159 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: ss_sold_date_sk (type: int), ss_cdemo_sk (type: int), ss_hdemo_sk (type: int), ss_addr_sk (type: int), ss_store_sk (type: int), ss_quantity (type: int), ss_sales_price (type: decimal(7,2)), ss_ext_sales_price (type: decimal(7,2)), ss_ext_wholesale_cost (type: decimal(7,2)), ss_net_profit (type: decimal(7,2))
-                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-                      Statistics: Num rows: 191998545 Data size: 16938167362 Basic stats: COMPLETE Column stats: NONE
+                      expressions: cd_demo_sk (type: int), cd_marital_status (type: string), cd_education_status (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 1861800 Data size: 717186159 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 191998545 Data size: 16938167362 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: decimal(7,2)), _col7 (type: decimal(7,2)), _col8 (type: decimal(7,2)), _col9 (type: decimal(7,2))
+                        Statistics: Num rows: 1861800 Data size: 717186159 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string), _col2 (type: string)
             Execution mode: vectorized
-        Map 6 
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  filterExpr: ((ss_sales_price BETWEEN 100 AND 150 or ss_sales_price BETWEEN 50 AND 100 or ss_sales_price BETWEEN 150 AND 200) and (ss_net_profit BETWEEN 100 AND 200 or ss_net_profit BETWEEN 150 AND 300 or ss_net_profit BETWEEN 50 AND 250) and ss_store_sk is not null and ss_cdemo_sk is not null and ss_hdemo_sk is not null and ss_addr_sk is not null and ss_sold_date_sk is not null) (type: boolean)
+                  Statistics: Num rows: 575995635 Data size: 50814502088 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((ss_net_profit BETWEEN 100 AND 200 or ss_net_profit BETWEEN 150 AND 300 or ss_net_profit BETWEEN 50 AND 250) and (ss_sales_price BETWEEN 100 AND 150 or ss_sales_price BETWEEN 50 AND 100 or ss_sales_price BETWEEN 150 AND 200) and ss_addr_sk is not null and ss_cdemo_sk is not null and ss_hdemo_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null) (type: boolean)
+                    Statistics: Num rows: 63999513 Data size: 5646055611 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ss_sold_date_sk (type: int), ss_cdemo_sk (type: int), ss_hdemo_sk (type: int), ss_addr_sk (type: int), ss_store_sk (type: int), ss_quantity (type: int), ss_sales_price (type: decimal(7,2)), ss_ext_sales_price (type: decimal(7,2)), ss_ext_wholesale_cost (type: decimal(7,2)), ss_net_profit (type: decimal(7,2))
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
+                      Statistics: Num rows: 63999513 Data size: 5646055611 Basic stats: COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col4 (type: int)
+                        outputColumnNames: _col1, _col2, _col3, _col4, _col6, _col7, _col8, _col9, _col10
+                        input vertices:
+                          0 Map 1
+                        Statistics: Num rows: 70399465 Data size: 6210661306 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: int)
+                          Statistics: Num rows: 70399465 Data size: 6210661306 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col2 (type: int), _col3 (type: int), _col4 (type: int), _col6 (type: int), _col7 (type: decimal(7,2)), _col8 (type: decimal(7,2)), _col9 (type: decimal(7,2)), _col10 (type: decimal(7,2))
+            Execution mode: vectorized
+            Local Work:
+              Map Reduce Local Work
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -217,34 +249,14 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 36524 Data size: 40870356 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: customer_demographics
-                  filterExpr: ((cd_marital_status) IN ('M', 'D', 'U') and (cd_education_status) IN ('4 yr Degree', 'Primary', 'Advanced Degree') and cd_demo_sk is not null) (type: boolean)
-                  Statistics: Num rows: 1861800 Data size: 717186159 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: ((cd_education_status) IN ('4 yr Degree', 'Primary', 'Advanced Degree') and (cd_marital_status) IN ('M', 'D', 'U') and cd_demo_sk is not null) (type: boolean)
-                    Statistics: Num rows: 1861800 Data size: 717186159 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: cd_demo_sk (type: int), cd_marital_status (type: string), cd_education_status (type: string)
-                      outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 1861800 Data size: 717186159 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 1861800 Data size: 717186159 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: string), _col2 (type: string)
-            Execution mode: vectorized
         Map 9 
             Map Operator Tree:
                 TableScan
                   alias: customer_address
-                  filterExpr: ((ca_country = 'United States') and ca_address_sk is not null) (type: boolean)
+                  filterExpr: ((ca_state) IN ('KY', 'GA', 'NM', 'MT', 'OR', 'IN', 'WI', 'MO', 'WV') and (ca_country = 'United States') and ca_address_sk is not null) (type: boolean)
                   Statistics: Num rows: 40000000 Data size: 40595195284 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: ((ca_country = 'United States') and ca_address_sk is not null) (type: boolean)
+                    predicate: ((ca_country = 'United States') and (ca_state) IN ('KY', 'GA', 'NM', 'MT', 'OR', 'IN', 'WI', 'MO', 'WV') and ca_address_sk is not null) (type: boolean)
                     Statistics: Num rows: 20000000 Data size: 20297597642 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: ca_address_sk (type: int), ca_state (type: string)
@@ -257,22 +269,6 @@ STAGE PLANS:
                         Statistics: Num rows: 20000000 Data size: 20297597642 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: string)
             Execution mode: vectorized
-        Reducer 2 
-            Reduce Operator Tree:
-              Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-                Statistics: Num rows: 211198404 Data size: 18631984502 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col1 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col1 (type: int)
-                  Statistics: Num rows: 211198404 Data size: 18631984502 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: decimal(7,2)), _col7 (type: decimal(7,2)), _col8 (type: decimal(7,2)), _col9 (type: decimal(7,2))
         Reducer 3 
             Local Work:
               Map Reduce Local Work
@@ -283,54 +279,62 @@ STAGE PLANS:
                 keys:
                   0 _col1 (type: int)
                   1 _col0 (type: int)
-                outputColumnNames: _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col13, _col14
-                Statistics: Num rows: 232318249 Data size: 20495183396 Basic stats: COMPLETE Column stats: NONE
+                outputColumnNames: _col2, _col3, _col4, _col6, _col7, _col8, _col9, _col10
+                Statistics: Num rows: 77439413 Data size: 6831727584 Basic stats: COMPLETE Column stats: NONE
                 Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col2 (type: int)
+                    0 _col3 (type: int)
                     1 _col0 (type: int)
-                  outputColumnNames: _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col13, _col14, _col16
+                  outputColumnNames: _col2, _col4, _col6, _col7, _col8, _col9, _col10, _col14
                   input vertices:
                     1 Map 8
-                  Statistics: Num rows: 255550079 Data size: 22544702224 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (((_col13 = 'D') and (_col14 = 'Primary') and _col6 BETWEEN 50 AND 100 and (_col16 = 1)) or ((_col13 = 'M') and (_col14 = '4 yr Degree') and _col6 BETWEEN 100 AND 150 and (_col16 = 3)) or ((_col13 = 'U') and (_col14 = 'Advanced Degree') and _col6 BETWEEN 150 AND 200 and (_col16 = 1))) (type: boolean)
-                    Statistics: Num rows: 10647918 Data size: 939362419 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col3 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col3 (type: int)
-                      Statistics: Num rows: 10647918 Data size: 939362419 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col4 (type: int), _col5 (type: int), _col7 (type: decimal(7,2)), _col8 (type: decimal(7,2)), _col9 (type: decimal(7,2))
+                  Statistics: Num rows: 85183356 Data size: 7514900505 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col4 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col4 (type: int)
+                    Statistics: Num rows: 85183356 Data size: 7514900505 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col2 (type: int), _col6 (type: int), _col7 (type: decimal(7,2)), _col8 (type: decimal(7,2)), _col9 (type: decimal(7,2)), _col10 (type: decimal(7,2)), _col14 (type: int)
         Reducer 4 
-            Local Work:
-              Map Reduce Local Work
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col3 (type: int)
+                  0 _col4 (type: int)
                   1 _col0 (type: int)
-                outputColumnNames: _col4, _col5, _col7, _col8, _col9, _col18
-                Statistics: Num rows: 22000000 Data size: 22327357890 Basic stats: COMPLETE Column stats: NONE
+                outputColumnNames: _col2, _col6, _col7, _col8, _col9, _col10, _col14, _col16
+                Statistics: Num rows: 93701693 Data size: 8266390734 Basic stats: COMPLETE Column stats: NONE
                 Filter Operator
-                  predicate: (((_col18) IN ('KY', 'GA', 'NM') and _col9 BETWEEN 100 AND 200) or ((_col18) IN ('MT', 'OR', 'IN') and _col9 BETWEEN 150 AND 300) or ((_col18) IN ('WI', 'MO', 'WV') and _col9 BETWEEN 50 AND 250)) (type: boolean)
-                  Statistics: Num rows: 7333332 Data size: 7442451276 Basic stats: COMPLETE Column stats: NONE
-                  Map Join Operator
-                    condition map:
-                         Inner Join 0 to 1
-                    keys:
-                      0 _col4 (type: int)
-                      1 _col0 (type: int)
-                    outputColumnNames: _col5, _col7, _col8
-                    input vertices:
-                      1 Map 10
-                    Statistics: Num rows: 8066665 Data size: 8186696581 Basic stats: COMPLETE Column stats: NONE
+                  predicate: (((_col16) IN ('KY', 'GA', 'NM') and _col10 BETWEEN 100 AND 200) or ((_col16) IN ('MT', 'OR', 'IN') and _col10 BETWEEN 150 AND 300) or ((_col16) IN ('WI', 'MO', 'WV') and _col10 BETWEEN 50 AND 250)) (type: boolean)
+                  Statistics: Num rows: 31233897 Data size: 2755463519 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col2 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col2 (type: int)
+                    Statistics: Num rows: 31233897 Data size: 2755463519 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col6 (type: int), _col7 (type: decimal(7,2)), _col8 (type: decimal(7,2)), _col9 (type: decimal(7,2)), _col14 (type: int)
+        Reducer 5 
+            Reduce Operator Tree:
+              Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col2 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col6, _col7, _col8, _col9, _col14, _col19, _col20
+                Statistics: Num rows: 34357287 Data size: 3031009936 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: (((_col19 = 'D') and (_col20 = 'Primary') and _col7 BETWEEN 50 AND 100 and (_col14 = 1)) or ((_col19 = 'M') and (_col20 = '4 yr Degree') and _col7 BETWEEN 100 AND 150 and (_col14 = 3)) or ((_col19 = 'U') and (_col20 = 'Advanced Degree') and _col7 BETWEEN 150 AND 200 and (_col14 = 1))) (type: boolean)
+                  Statistics: Num rows: 1431552 Data size: 126291937 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col6 (type: int), _col8 (type: decimal(7,2)), _col9 (type: decimal(7,2))
+                    outputColumnNames: _col6, _col8, _col9
+                    Statistics: Num rows: 1431552 Data size: 126291937 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col5), count(_col5), sum(_col7), count(_col7), sum(_col8), count(_col8)
+                      aggregations: sum(_col6), count(_col6), sum(_col8), count(_col8), sum(_col9), count(_col9)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                       Statistics: Num rows: 1 Data size: 256 Basic stats: COMPLETE Column stats: NONE
@@ -338,7 +342,7 @@ STAGE PLANS:
                         sort order: 
                         Statistics: Num rows: 1 Data size: 256 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: decimal(17,2)), _col3 (type: bigint), _col4 (type: decimal(17,2)), _col5 (type: bigint)
-        Reducer 5 
+        Reducer 6 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/f0b76e24/ql/src/test/results/clientpositive/perf/spark/query47.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/spark/query47.q.out b/ql/src/test/results/clientpositive/perf/spark/query47.q.out
index 44665fb..f6a2e1b 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query47.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query47.q.out
@@ -255,10 +255,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: date_dim
-                  filterExpr: (((struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not null) (type: boolean)
+                  filterExpr: ((d_year) IN (2000, 1999, 2001) and ((d_year = 2000) or (struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1))) and d_date_sk is not null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not null) (type: boolean)
+                    predicate: (((d_year = 2000) or (struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1))) and (d_year) IN (2000, 1999, 2001) and d_date_sk is not null) (type: boolean)
                     Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_year (type: int), d_moy (type: int)
@@ -315,10 +315,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: date_dim
-                  filterExpr: (((struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not null) (type: boolean)
+                  filterExpr: ((d_year) IN (2000, 1999, 2001) and ((d_year = 2000) or (struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1))) and d_date_sk is not null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not null) (type: boolean)
+                    predicate: (((d_year = 2000) or (struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1))) and (d_year) IN (2000, 1999, 2001) and d_date_sk is not null) (type: boolean)
                     Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_year (type: int), d_moy (type: int)
@@ -355,10 +355,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: date_dim
-                  filterExpr: (((struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not null) (type: boolean)
+                  filterExpr: ((d_year) IN (2000, 1999, 2001) and ((d_year = 2000) or (struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1))) and d_date_sk is not null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not null) (type: boolean)
+                    predicate: (((d_year = 2000) or (struct(d_year,d_moy)) IN (const struct(1999,12), const struct(2001,1))) and (d_year) IN (2000, 1999, 2001) and d_date_sk is not null) (type: boolean)
                     Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_year (type: int), d_moy (type: int)
@@ -431,14 +431,14 @@ STAGE PLANS:
                   Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: sum(_col3)
-                    keys: _col5 (type: int), _col6 (type: int), _col8 (type: string), _col9 (type: string), _col11 (type: string), _col12 (type: string)
+                    keys: _col8 (type: string), _col9 (type: string), _col5 (type: int), _col6 (type: int), _col11 (type: string), _col12 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                     Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                      key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: string), _col5 (type: string)
                       sort order: ++++++
-                      Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                      Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: string), _col5 (type: string)
                       Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col6 (type: decimal(17,2))
         Reducer 14 
@@ -446,34 +446,34 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: sum(VALUE._col0)
-                keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: string), KEY._col5 (type: string)
+                keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: int), KEY._col3 (type: int), KEY._col4 (type: string), KEY._col5 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col3 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: string), _col0 (type: int)
+                  key expressions: _col1 (type: string), _col0 (type: string), _col4 (type: string), _col5 (type: string), _col2 (type: int)
                   sort order: +++++
-                  Map-reduce partition columns: _col3 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: string), _col0 (type: int)
+                  Map-reduce partition columns: _col1 (type: string), _col0 (type: string), _col4 (type: string), _col5 (type: string), _col2 (type: int)
                   Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col1 (type: int), _col6 (type: decimal(17,2))
+                  value expressions: _col3 (type: int), _col6 (type: decimal(17,2))
         Reducer 15 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey4 (type: int), VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), VALUE._col1 (type: decimal(17,2))
+                expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey4 (type: int), VALUE._col0 (type: int), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), VALUE._col1 (type: decimal(17,2))
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                 PTF Operator
                   Function definitions:
                       Input definition
                         input alias: ptf_0
-                        output shape: _col0: int, _col1: int, _col2: string, _col3: string, _col4: string, _col5: string, _col6: decimal(17,2)
+                        output shape: _col0: string, _col1: string, _col2: int, _col3: int, _col4: string, _col5: string, _col6: decimal(17,2)
                         type: WINDOWING
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col3 ASC NULLS FIRST, _col2 ASC NULLS FIRST, _col4 ASC NULLS FIRST, _col5 ASC NULLS FIRST, _col0 ASC NULLS FIRST
-                        partition by: _col3, _col2, _col4, _col5, _col0
+                        order by: _col1 ASC NULLS FIRST, _col0 ASC NULLS FIRST, _col4 ASC NULLS FIRST, _col5 ASC NULLS FIRST, _col2 ASC NULLS FIRST
+                        partition by: _col1, _col0, _col4, _col5, _col2
                         raw input shape:
                         window functions:
                             window function definition
@@ -484,55 +484,55 @@ STAGE PLANS:
                               window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
                   Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: avg_window_0 (type: decimal(21,6)), _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: decimal(17,2))
+                    expressions: avg_window_0 (type: decimal(21,6)), _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: string), _col5 (type: string), _col6 (type: decimal(17,2))
                     outputColumnNames: avg_window_0, _col0, _col1, _col2, _col3, _col4, _col5, _col6
                     Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col3 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: string), _col0 (type: int), _col1 (type: int)
+                      key expressions: _col1 (type: string), _col0 (type: string), _col4 (type: string), _col5 (type: string), _col2 (type: int), _col3 (type: int)
                       sort order: ++++++
-                      Map-reduce partition columns: _col3 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: string)
+                      Map-reduce partition columns: _col1 (type: string), _col0 (type: string), _col4 (type: string), _col5 (type: string)
                       Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                       value expressions: avg_window_0 (type: decimal(21,6)), _col6 (type: decimal(17,2))
         Reducer 16 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: decimal(21,6)), KEY.reducesinkkey4 (type: int), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), VALUE._col1 (type: decimal(17,2))
+                expressions: VALUE._col0 (type: decimal(21,6)), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey4 (type: int), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), VALUE._col1 (type: decimal(17,2))
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                 Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                 PTF Operator
                   Function definitions:
                       Input definition
                         input alias: ptf_0
-                        output shape: _col0: decimal(21,6), _col1: int, _col2: int, _col3: string, _col4: string, _col5: string, _col6: string, _col7: decimal(17,2)
+                        output shape: _col0: decimal(21,6), _col1: string, _col2: string, _col3: int, _col4: int, _col5: string, _col6: string, _col7: decimal(17,2)
                         type: WINDOWING
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1 ASC NULLS LAST, _col2 ASC NULLS LAST
-                        partition by: _col4, _col3, _col5, _col6
+                        order by: _col3 ASC NULLS LAST, _col4 ASC NULLS LAST
+                        partition by: _col2, _col1, _col5, _col6
                         raw input shape:
                         window functions:
                             window function definition
                               alias: rank_window_1
-                              arguments: _col1, _col2
+                              arguments: _col3, _col4
                               name: rank
                               window function: GenericUDAFRankEvaluator
                               window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
                               isPivotResult: true
                   Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: ((_col0 > 0) and (_col1 = 2000) and rank_window_1 is not null) (type: boolean)
+                    predicate: ((_col0 > 0) and (_col3 = 2000) and rank_window_1 is not null) (type: boolean)
                     Statistics: Num rows: 63887519 Data size: 5636175475 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: rank_window_1 (type: int), _col0 (type: decimal(21,6)), _col1 (type: int), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: decimal(17,2))
+                      expressions: rank_window_1 (type: int), _col0 (type: decimal(21,6)), _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string), _col6 (type: string), _col7 (type: decimal(17,2))
                       outputColumnNames: rank_window_1, _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                       Statistics: Num rows: 63887519 Data size: 5636175475 Basic stats: COMPLETE Column stats: NONE
                       Filter Operator
                         predicate: CASE WHEN ((_col0 > 0)) THEN (((abs((_col7 - _col0)) / _col0) > 0.1)) ELSE (null) END (type: boolean)
                         Statistics: Num rows: 31943759 Data size: 2818087693 Basic stats: COMPLETE Column stats: NONE
                         Select Operator
-                          expressions: _col4 (type: string), _col3 (type: string), _col5 (type: string), _col6 (type: string), _col1 (type: int), _col2 (type: int), _col7 (type: decimal(17,2)), _col0 (type: decimal(21,6)), rank_window_1 (type: int)
+                          expressions: _col2 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col3 (type: int), _col4 (type: int), _col7 (type: decimal(17,2)), _col0 (type: decimal(21,6)), rank_window_1 (type: int)
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                           Statistics: Num rows: 31943759 Data size: 2818087693 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
@@ -597,14 +597,14 @@ STAGE PLANS:
                   Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: sum(_col3)
-                    keys: _col5 (type: int), _col6 (type: int), _col8 (type: string), _col9 (type: string), _col11 (type: string), _col12 (type: string)
+                    keys: _col8 (type: string), _col9 (type: string), _col5 (type: int), _col6 (type: int), _col11 (type: string), _col12 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                     Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                      key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: string), _col5 (type: string)
                       sort order: ++++++
-                      Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                      Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: string), _col5 (type: string)
                       Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col6 (type: decimal(17,2))
         Reducer 23 
@@ -612,39 +612,39 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: sum(VALUE._col0)
-                keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: string), KEY._col5 (type: string)
+                keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: int), KEY._col3 (type: int), KEY._col4 (type: string), KEY._col5 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col3 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: string), _col0 (type: int), _col1 (type: int)
+                  key expressions: _col1 (type: string), _col0 (type: string), _col4 (type: string), _col5 (type: string), _col2 (type: int), _col3 (type: int)
                   sort order: ++++++
-                  Map-reduce partition columns: _col3 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: string)
+                  Map-reduce partition columns: _col1 (type: string), _col0 (type: string), _col4 (type: string), _col5 (type: string)
                   Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col6 (type: decimal(17,2))
         Reducer 24 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey4 (type: int), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), VALUE._col0 (type: decimal(17,2))
+                expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey4 (type: int), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), VALUE._col0 (type: decimal(17,2))
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                 PTF Operator
                   Function definitions:
                       Input definition
                         input alias: ptf_0
-                        output shape: _col0: int, _col1: int, _col2: string, _col3: string, _col4: string, _col5: string, _col6: decimal(17,2)
+                        output shape: _col0: string, _col1: string, _col2: int, _col3: int, _col4: string, _col5: string, _col6: decimal(17,2)
                         type: WINDOWING
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col0 ASC NULLS LAST, _col1 ASC NULLS LAST
-                        partition by: _col3, _col2, _col4, _col5
+                        order by: _col2 ASC NULLS LAST, _col3 ASC NULLS LAST
+                        partition by: _col1, _col0, _col4, _col5
                         raw input shape:
                         window functions:
                             window function definition
                               alias: rank_window_0
-                              arguments: _col0, _col1
+                              arguments: _col2, _col3
                               name: rank
                               window function: GenericUDAFRankEvaluator
                               window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
@@ -654,7 +654,7 @@ STAGE PLANS:
                     predicate: rank_window_0 is not null (type: boolean)
                     Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: _col3 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: decimal(17,2)), rank_window_0 (type: int)
+                      expressions: _col1 (type: string), _col0 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: decimal(17,2)), rank_window_0 (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                       Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
@@ -687,14 +687,14 @@ STAGE PLANS:
                   Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: sum(_col3)
-                    keys: _col5 (type: int), _col6 (type: int), _col8 (type: string), _col9 (type: string), _col11 (type: string), _col12 (type: string)
+                    keys: _col8 (type: string), _col9 (type: string), _col5 (type: int), _col6 (type: int), _col11 (type: string), _col12 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                     Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
-                      key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                      key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: string), _col5 (type: string)
                       sort order: ++++++
-                      Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                      Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: string), _col5 (type: string)
                       Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col6 (type: decimal(17,2))
         Reducer 4 
@@ -702,39 +702,39 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: sum(VALUE._col0)
-                keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: string), KEY._col5 (type: string)
+                keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: int), KEY._col3 (type: int), KEY._col4 (type: string), KEY._col5 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col3 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: string), _col0 (type: int), _col1 (type: int)
+                  key expressions: _col1 (type: string), _col0 (type: string), _col4 (type: string), _col5 (type: string), _col2 (type: int), _col3 (type: int)
                   sort order: ++++++
-                  Map-reduce partition columns: _col3 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: string)
+                  Map-reduce partition columns: _col1 (type: string), _col0 (type: string), _col4 (type: string), _col5 (type: string)
                   Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col6 (type: decimal(17,2))
         Reducer 5 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey4 (type: int), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), VALUE._col0 (type: decimal(17,2))
+                expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey4 (type: int), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), VALUE._col0 (type: decimal(17,2))
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                 PTF Operator
                   Function definitions:
                       Input definition
                         input alias: ptf_0
-                        output shape: _col0: int, _col1: int, _col2: string, _col3: string, _col4: string, _col5: string, _col6: decimal(17,2)
+                        output shape: _col0: string, _col1: string, _col2: int, _col3: int, _col4: string, _col5: string, _col6: decimal(17,2)
                         type: WINDOWING
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col0 ASC NULLS LAST, _col1 ASC NULLS LAST
-                        partition by: _col3, _col2, _col4, _col5
+                        order by: _col2 ASC NULLS LAST, _col3 ASC NULLS LAST
+                        partition by: _col1, _col0, _col4, _col5
                         raw input shape:
                         window functions:
                             window function definition
                               alias: rank_window_0
-                              arguments: _col0, _col1
+                              arguments: _col2, _col3
                               name: rank
                               window function: GenericUDAFRankEvaluator
                               window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
@@ -744,7 +744,7 @@ STAGE PLANS:
                     predicate: rank_window_0 is not null (type: boolean)
                     Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: _col3 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: decimal(17,2)), rank_window_0 (type: int)
+                      expressions: _col1 (type: string), _col0 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: decimal(17,2)), rank_window_0 (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                       Statistics: Num rows: 383325119 Data size: 33817053293 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/f0b76e24/ql/src/test/results/clientpositive/perf/spark/query48.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/spark/query48.q.out b/ql/src/test/results/clientpositive/perf/spark/query48.q.out
index 24f5425..a775c7e 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query48.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query48.q.out
@@ -150,7 +150,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 9 
+        Map 1 
             Map Operator Tree:
                 TableScan
                   alias: store
@@ -165,8 +165,8 @@ STAGE PLANS:
                       Statistics: Num rows: 1704 Data size: 3256276 Basic stats: COMPLETE Column stats: NONE
                       Spark HashTable Sink Operator
                         keys:
-                          0 _col3 (type: int)
-                          1 _col0 (type: int)
+                          0 _col0 (type: int)
+                          1 _col3 (type: int)
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
@@ -174,33 +174,45 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 133), Map 6 (PARTITION-LEVEL SORT, 133)
-        Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 147), Reducer 2 (PARTITION-LEVEL SORT, 147)
-        Reducer 4 <- Map 8 (PARTITION-LEVEL SORT, 319), Reducer 3 (PARTITION-LEVEL SORT, 319)
-        Reducer 5 <- Reducer 4 (GROUP, 1)
+        Reducer 3 <- Map 2 (PARTITION-LEVEL SORT, 49), Map 7 (PARTITION-LEVEL SORT, 49)
+        Reducer 4 <- Map 8 (PARTITION-LEVEL SORT, 55), Reducer 3 (PARTITION-LEVEL SORT, 55)
+        Reducer 5 <- Map 9 (PARTITION-LEVEL SORT, 218), Reducer 4 (PARTITION-LEVEL SORT, 218)
+        Reducer 6 <- Reducer 5 (GROUP, 1)
 #### A masked pattern was here ####
       Vertices:
-        Map 1 
+        Map 2 
             Map Operator Tree:
                 TableScan
                   alias: store_sales
-                  filterExpr: ((ss_sales_price BETWEEN 100 AND 150 or ss_sales_price BETWEEN 50 AND 100 or ss_sales_price BETWEEN 150 AND 200) and ss_store_sk is not null and ss_cdemo_sk is not null and ss_addr_sk is not null and ss_sold_date_sk is not null) (type: boolean)
+                  filterExpr: ((ss_sales_price BETWEEN 100 AND 150 or ss_sales_price BETWEEN 50 AND 100 or ss_sales_price BETWEEN 150 AND 200) and (ss_net_profit BETWEEN 0 AND 2000 or ss_net_profit BETWEEN 150 AND 3000 or ss_net_profit BETWEEN 50 AND 25000) and ss_store_sk is not null and ss_cdemo_sk is not null and ss_addr_sk is not null and ss_sold_date_sk is not null) (type: boolean)
                   Statistics: Num rows: 575995635 Data size: 50814502088 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: ((ss_sales_price BETWEEN 100 AND 150 or ss_sales_price BETWEEN 50 AND 100 or ss_sales_price BETWEEN 150 AND 200) and ss_addr_sk is not null and ss_cdemo_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null) (type: boolean)
-                    Statistics: Num rows: 191998545 Data size: 16938167362 Basic stats: COMPLETE Column stats: NONE
+                    predicate: ((ss_net_profit BETWEEN 0 AND 2000 or ss_net_profit BETWEEN 150 AND 3000 or ss_net_profit BETWEEN 50 AND 25000) and (ss_sales_price BETWEEN 100 AND 150 or ss_sales_price BETWEEN 50 AND 100 or ss_sales_price BETWEEN 150 AND 200) and ss_addr_sk is not null and ss_cdemo_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null) (type: boolean)
+                    Statistics: Num rows: 63999513 Data size: 5646055611 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: ss_sold_date_sk (type: int), ss_cdemo_sk (type: int), ss_addr_sk (type: int), ss_store_sk (type: int), ss_quantity (type: int), ss_net_profit (type: decimal(7,2))
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6
-                      Statistics: Num rows: 191998545 Data size: 16938167362 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 191998545 Data size: 16938167362 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col6 (type: decimal(7,2))
+                      Statistics: Num rows: 63999513 Data size: 5646055611 Basic stats: COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col3 (type: int)
+                        outputColumnNames: _col1, _col2, _col3, _col5, _col7
+                        input vertices:
+                          0 Map 1
+                        Statistics: Num rows: 70399465 Data size: 6210661306 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: int)
+                          Statistics: Num rows: 70399465 Data size: 6210661306 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col2 (type: int), _col3 (type: int), _col5 (type: int), _col7 (type: decimal(7,2))
             Execution mode: vectorized
-        Map 6 
+            Local Work:
+              Map Reduce Local Work
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -219,7 +231,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 36524 Data size: 40870356 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 7 
+        Map 8 
             Map Operator Tree:
                 TableScan
                   alias: customer_demographics
@@ -238,14 +250,14 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 465450 Data size: 179296539 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 8 
+        Map 9 
             Map Operator Tree:
                 TableScan
                   alias: customer_address
-                  filterExpr: ((ca_country = 'United States') and ca_address_sk is not null) (type: boolean)
+                  filterExpr: ((ca_state) IN ('KY', 'GA', 'NM', 'MT', 'OR', 'IN', 'WI', 'MO', 'WV') and (ca_country = 'United States') and ca_address_sk is not null) (type: boolean)
                   Statistics: Num rows: 40000000 Data size: 40595195284 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: ((ca_country = 'United States') and ca_address_sk is not null) (type: boolean)
+                    predicate: ((ca_country = 'United States') and (ca_state) IN ('KY', 'GA', 'NM', 'MT', 'OR', 'IN', 'WI', 'MO', 'WV') and ca_address_sk is not null) (type: boolean)
                     Statistics: Num rows: 20000000 Data size: 20297597642 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: ca_address_sk (type: int), ca_state (type: string)
@@ -258,65 +270,57 @@ STAGE PLANS:
                         Statistics: Num rows: 20000000 Data size: 20297597642 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: string)
             Execution mode: vectorized
-        Reducer 2 
+        Reducer 3 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
+                  0 _col1 (type: int)
                   1 _col0 (type: int)
-                outputColumnNames: _col1, _col2, _col3, _col4, _col6
-                Statistics: Num rows: 211198404 Data size: 18631984502 Basic stats: COMPLETE Column stats: NONE
+                outputColumnNames: _col2, _col3, _col5, _col7
+                Statistics: Num rows: 77439413 Data size: 6831727584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col1 (type: int)
+                  key expressions: _col2 (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col1 (type: int)
-                  Statistics: Num rows: 211198404 Data size: 18631984502 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col2 (type: int), _col3 (type: int), _col4 (type: int), _col6 (type: decimal(7,2))
-        Reducer 3 
+                  Map-reduce partition columns: _col2 (type: int)
+                  Statistics: Num rows: 77439413 Data size: 6831727584 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col3 (type: int), _col5 (type: int), _col7 (type: decimal(7,2))
+        Reducer 4 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: int)
+                  0 _col2 (type: int)
                   1 _col0 (type: int)
-                outputColumnNames: _col2, _col3, _col4, _col6
-                Statistics: Num rows: 232318249 Data size: 20495183396 Basic stats: COMPLETE Column stats: NONE
+                outputColumnNames: _col3, _col5, _col7
+                Statistics: Num rows: 85183356 Data size: 7514900505 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col2 (type: int)
+                  key expressions: _col3 (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col2 (type: int)
-                  Statistics: Num rows: 232318249 Data size: 20495183396 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col3 (type: int), _col4 (type: int), _col6 (type: decimal(7,2))
-        Reducer 4 
-            Local Work:
-              Map Reduce Local Work
+                  Map-reduce partition columns: _col3 (type: int)
+                  Statistics: Num rows: 85183356 Data size: 7514900505 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col5 (type: int), _col7 (type: decimal(7,2))
+        Reducer 5 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col2 (type: int)
+                  0 _col3 (type: int)
                   1 _col0 (type: int)
-                outputColumnNames: _col3, _col4, _col6, _col13
-                Statistics: Num rows: 255550079 Data size: 22544702224 Basic stats: COMPLETE Column stats: NONE
+                outputColumnNames: _col5, _col7, _col14
+                Statistics: Num rows: 93701693 Data size: 8266390734 Basic stats: COMPLETE Column stats: NONE
                 Filter Operator
-                  predicate: (((_col13) IN ('KY', 'GA', 'NM') and _col6 BETWEEN 0 AND 2000) or ((_col13) IN ('MT', 'OR', 'IN') and _col6 BETWEEN 150 AND 3000) or ((_col13) IN ('WI', 'MO', 'WV') and _col6 BETWEEN 50 AND 25000)) (type: boolean)
-                  Statistics: Num rows: 85183359 Data size: 7514900682 Basic stats: COMPLETE Column stats: NONE
-                  Map Join Operator
-                    condition map:
-                         Inner Join 0 to 1
-                    keys:
-                      0 _col3 (type: int)
-                      1 _col0 (type: int)
-                    outputColumnNames: _col4
-                    input vertices:
-                      1 Map 9
-                    Statistics: Num rows: 93701696 Data size: 8266390929 Basic stats: COMPLETE Column stats: NONE
+                  predicate: (((_col14) IN ('KY', 'GA', 'NM') and _col7 BETWEEN 0 AND 2000) or ((_col14) IN ('MT', 'OR', 'IN') and _col7 BETWEEN 150 AND 3000) or ((_col14) IN ('WI', 'MO', 'WV') and _col7 BETWEEN 50 AND 25000)) (type: boolean)
+                  Statistics: Num rows: 31233897 Data size: 2755463519 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col5 (type: int)
+                    outputColumnNames: _col5
+                    Statistics: Num rows: 31233897 Data size: 2755463519 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: sum(_col4)
+                      aggregations: sum(_col5)
                       mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
@@ -324,7 +328,7 @@ STAGE PLANS:
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
-        Reducer 5 
+        Reducer 6 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/f0b76e24/ql/src/test/results/clientpositive/perf/spark/query53.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/spark/query53.q.out b/ql/src/test/results/clientpositive/perf/spark/query53.q.out
index 34593b7..3479cb5 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query53.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query53.q.out
@@ -126,10 +126,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: item
-                  filterExpr: ((((i_category) IN ('Books', 'Children', 'Electronics') and (i_class) IN ('personal', 'portable', 'reference', 'self-help') and (i_brand) IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) or ((i_category) IN ('Women', 'Music', 'Men') and (i_class) IN ('accessories', 'classical', 'fragrances', 'pants') and (i_brand) IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_item_sk is not null) (type: boolean)
+                  filterExpr: ((i_class) IN ('personal', 'portable', 'reference', 'self-help', 'accessories', 'classical', 'fragrances', 'pants') and (i_brand) IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9', 'amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1') and (i_category) IN ('Books', 'Children', 'Electronics', 'Women', 'Music', 'Men') and (((i_category) IN ('Books', 'Children', 'Electronics') and (i_class) IN ('personal', 'portable', 'reference', 'self-help') and (i_brand) IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) or ((i_category) IN ('Women', 'Music', 'Men') and (i_class) IN ('accessories', 'classical', 'fragrances', 'pants') and (i_brand) IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_item_sk is not null) (type: boolean)
                   Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: ((((i_category) IN ('Books', 'Children', 'Electronics') and (i_class) IN ('personal', 'portable', 'reference', 'self-help') and (i_brand) IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) or ((i_category) IN ('Women', 'Music', 'Men') and (i_class) IN ('accessories', 'classical', 'fragrances', 'pants') and (i_brand) IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_item_sk is not null) (type: boolean)
+                    predicate: ((((i_category) IN ('Books', 'Children', 'Electronics') and (i_class) IN ('personal', 'portable', 'reference', 'self-help') and (i_brand) IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) or ((i_category) IN ('Women', 'Music', 'Men') and (i_class) IN ('accessories', 'classical', 'fragrances', 'pants') and (i_brand) IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and (i_brand) IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9', 'amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1') and (i_category) IN ('Books', 'Children', 'Electronics', 'Women', 'Music', 'Men') and (i_class) IN ('personal', 'portable', 'reference', 'self-help', 'accessories', 'classical', 'fragrances', 'pants') and i_item_sk is not null) (type: boolean)
                     Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: i_item_sk (type: int), i_manufact_id (type: int)