You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sp...@apache.org on 2016/05/06 20:43:02 UTC

[31/50] [abbrv] hive git commit: HIVE-13639: CBO rule to pull up constants through Union (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

HIVE-13639: CBO rule to pull up constants through Union (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/09271872
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/09271872
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/09271872

Branch: refs/heads/java8
Commit: 092718720a4abc77ce74c2efcf42cfef0243e9d4
Parents: f41d693
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed May 4 22:01:52 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu May 5 20:21:50 2016 +0100

----------------------------------------------------------------------
 .../rules/HiveUnionPullUpConstantsRule.java     | 133 ++++
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   2 +
 .../queries/clientpositive/cbo_union_view.q     |  19 +
 .../results/clientpositive/cbo_input26.q.out    |  64 +-
 .../results/clientpositive/cbo_union_view.q.out | 228 ++++++
 .../results/clientpositive/groupby_ppd.q.out    |  28 +-
 .../results/clientpositive/perf/query66.q.out   | 328 ++++-----
 .../results/clientpositive/perf/query75.q.out   | 692 ++++++++++---------
 .../clientpositive/spark/union_remove_25.q.out  |  48 +-
 .../clientpositive/spark/union_view.q.out       |  60 +-
 .../results/clientpositive/union_view.q.out     |  60 +-
 11 files changed, 1021 insertions(+), 641 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java
new file mode 100644
index 0000000..3155cb1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Planner rule that pulls up constants through a Union operator.
+ */
+public class HiveUnionPullUpConstantsRule extends RelOptRule {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(HiveUnionPullUpConstantsRule.class);
+
+
+  public static final HiveUnionPullUpConstantsRule INSTANCE =
+          new HiveUnionPullUpConstantsRule(HiveUnion.class,
+                  HiveRelFactories.HIVE_BUILDER);
+
+  private HiveUnionPullUpConstantsRule(
+      Class<? extends Union> unionClass,
+      RelBuilderFactory relBuilderFactory) {
+    super(operand(unionClass, any()),
+            relBuilderFactory, null);
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final Union union = call.rel(0);
+
+    final int count = union.getRowType().getFieldCount();
+    if (count == 1) {
+      // No room for optimization since we cannot create an empty
+      // Project operator.
+      return;
+    }
+
+    final RexBuilder rexBuilder = union.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final RelOptPredicateList predicates = mq.getPulledUpPredicates(union);
+    if (predicates == null) {
+      return;
+    }
+
+    Map<RexNode, RexNode> constants = HiveReduceExpressionsRule.predicateConstants(
+            RexNode.class, rexBuilder, predicates);
+
+    // None of the expressions are constant. Nothing to do.
+    if (constants.isEmpty()) {
+      return;
+    }
+
+    if (count == constants.size()) {
+      // At least a single item in project is required.
+      final Map<RexNode, RexNode> map = new HashMap<>(constants);
+      map.remove(map.keySet().iterator().next());
+      constants = map;
+    }
+
+    // Create expressions for Project operators before and after the Union
+    List<RelDataTypeField> fields = union.getRowType().getFieldList();
+    List<Pair<RexNode, String>> newChildExprs = new ArrayList<>();
+    List<RexNode> topChildExprs = new ArrayList<>();
+    List<String> topChildExprsFields = new ArrayList<>();
+    for (int i = 0; i < count ; i++) {
+      RexNode expr = rexBuilder.makeInputRef(union, i);
+      RelDataTypeField field = fields.get(i);
+      if (constants.containsKey(expr)) {
+        topChildExprs.add(constants.get(expr));
+        topChildExprsFields.add(field.getName());
+      } else {
+        newChildExprs.add(Pair.<RexNode,String>of(expr, field.getName()));
+        topChildExprs.add(expr);
+        topChildExprsFields.add(field.getName());
+      }
+    }
+
+    // Update top Project positions
+    final Mappings.TargetMapping mapping =
+            RelOptUtil.permutation(Pair.left(newChildExprs), union.getInput(0).getRowType()).inverse();
+    topChildExprs = ImmutableList.copyOf(RexUtil.apply(mapping, topChildExprs));
+
+    // Create new Project-Union-Project sequences
+    final RelBuilder relBuilder = call.builder();
+    for (int i = 0; i < union.getInputs().size() ; i++) {
+      relBuilder.push(union.getInput(i));
+      relBuilder.project(Pair.left(newChildExprs), Pair.right(newChildExprs));
+    }
+    relBuilder.union(union.all, union.getInputs().size());
+    relBuilder.project(topChildExprs, topChildExprsFields);
+
+    call.transformTo(relBuilder.build());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 377573b..de6a053 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -154,6 +154,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinProjectTranspos
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinPushTransitivePredicatesRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinToMultiJoinRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortLimitPullUpConstantsRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveUnionPullUpConstantsRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePartitionPruneRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePointLookupOptimizerRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePreFilteringRule;
@@ -1166,6 +1167,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_SEMIJOIN);
       rules.add(HiveSortMergeRule.INSTANCE);
       rules.add(HiveSortLimitPullUpConstantsRule.INSTANCE);
+      rules.add(HiveUnionPullUpConstantsRule.INSTANCE);
       perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
       basePlan = hepPlan(basePlan, true, mdProvider, executorProvider, HepMatchOrder.BOTTOM_UP,
               rules.toArray(new RelOptRule[rules.size()]));

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/queries/clientpositive/cbo_union_view.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_union_view.q b/ql/src/test/queries/clientpositive/cbo_union_view.q
new file mode 100644
index 0000000..d889b1d
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/cbo_union_view.q
@@ -0,0 +1,19 @@
+set hive.mapred.mode=nonstrict;
+set hive.optimize.constant.propagation=false;
+
+CREATE TABLE src_union_1 (key int, value string) PARTITIONED BY (ds string);
+CREATE TABLE src_union_2 (key int, value string) PARTITIONED BY (ds string, part_1 string);
+CREATE TABLE src_union_3 (key int, value string) PARTITIONED BY (ds string, part_1 string, part_2 string);
+
+CREATE VIEW src_union_view PARTITIONED ON (ds) as
+SELECT key, value, ds FROM (
+SELECT key, value, ds FROM src_union_1
+UNION ALL
+SELECT key, value, ds FROM src_union_2
+UNION ALL
+SELECT key, value, ds FROM src_union_3
+) subq;
+
+EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86;
+
+EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='1';

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/cbo_input26.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_input26.q.out b/ql/src/test/results/clientpositive/cbo_input26.q.out
index 5c4c771..c9ed67a 100644
--- a/ql/src/test/results/clientpositive/cbo_input26.q.out
+++ b/ql/src/test/results/clientpositive/cbo_input26.q.out
@@ -490,16 +490,12 @@ STAGE PLANS:
           Limit
             Number of rows: 5
             Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: _col0 (type: string), '2008-04-08' (type: string), _col1 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
-                table:
-                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-2
     Map Reduce
@@ -507,23 +503,31 @@ STAGE PLANS:
           TableScan
             Union
               Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
+              Select Operator
+                expressions: _col0 (type: string), '2008-04-08' (type: string), _col1 (type: string)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 50 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
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 6 Data size: 50 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
           TableScan
             Union
               Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
+              Select Operator
+                expressions: _col0 (type: string), '2008-04-08' (type: string), _col1 (type: string)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 50 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
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 6 Data size: 50 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-3
     Map Reduce
@@ -554,16 +558,12 @@ STAGE PLANS:
           Limit
             Number of rows: 5
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-            Select Operator
-              expressions: _col0 (type: string), '2008-04-08' (type: string), _col1 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-              File Output Operator
-                compressed: false
-                table:
-                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/cbo_union_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_union_view.q.out b/ql/src/test/results/clientpositive/cbo_union_view.q.out
new file mode 100644
index 0000000..ed6bba9
--- /dev/null
+++ b/ql/src/test/results/clientpositive/cbo_union_view.q.out
@@ -0,0 +1,228 @@
+PREHOOK: query: CREATE TABLE src_union_1 (key int, value string) PARTITIONED BY (ds string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_union_1
+POSTHOOK: query: CREATE TABLE src_union_1 (key int, value string) PARTITIONED BY (ds string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_union_1
+PREHOOK: query: CREATE TABLE src_union_2 (key int, value string) PARTITIONED BY (ds string, part_1 string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_union_2
+POSTHOOK: query: CREATE TABLE src_union_2 (key int, value string) PARTITIONED BY (ds string, part_1 string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_union_2
+PREHOOK: query: CREATE TABLE src_union_3 (key int, value string) PARTITIONED BY (ds string, part_1 string, part_2 string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_union_3
+POSTHOOK: query: CREATE TABLE src_union_3 (key int, value string) PARTITIONED BY (ds string, part_1 string, part_2 string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_union_3
+PREHOOK: query: CREATE VIEW src_union_view PARTITIONED ON (ds) as
+SELECT key, value, ds FROM (
+SELECT key, value, ds FROM src_union_1
+UNION ALL
+SELECT key, value, ds FROM src_union_2
+UNION ALL
+SELECT key, value, ds FROM src_union_3
+) subq
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: default@src_union_1
+PREHOOK: Input: default@src_union_2
+PREHOOK: Input: default@src_union_3
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_union_view
+POSTHOOK: query: CREATE VIEW src_union_view PARTITIONED ON (ds) as
+SELECT key, value, ds FROM (
+SELECT key, value, ds FROM src_union_1
+UNION ALL
+SELECT key, value, ds FROM src_union_2
+UNION ALL
+SELECT key, value, ds FROM src_union_3
+) subq
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: default@src_union_1
+POSTHOOK: Input: default@src_union_2
+POSTHOOK: Input: default@src_union_3
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_union_view
+PREHOOK: query: EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src_union_1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (key = 86) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), _col1 (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL 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
+          TableScan
+            alias: src_union_2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (key = 86) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), _col1 (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL 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
+          TableScan
+            alias: src_union_3
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (key = 86) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), _col1 (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL 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
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='1'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='1'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src_union_1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((key = 86) and (ds = '1')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL 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
+          TableScan
+            alias: src_union_2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((key = 86) and (ds = '1')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL 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
+          TableScan
+            alias: src_union_3
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((key = 86) and (ds = '1')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL 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
+      limit: -1
+      Processor Tree:
+        ListSink
+

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/groupby_ppd.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_ppd.q.out b/ql/src/test/results/clientpositive/groupby_ppd.q.out
index c63acd3..515f62e 100644
--- a/ql/src/test/results/clientpositive/groupby_ppd.q.out
+++ b/ql/src/test/results/clientpositive/groupby_ppd.q.out
@@ -28,23 +28,23 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: foo (type: int)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
-                    expressions: 1 (type: int), _col1 (type: int)
-                    outputColumnNames: _col0, _col1
+                    expressions: _col0 (type: int)
+                    outputColumnNames: _col1
                     Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int), _col1 (type: int)
+                      keys: 1 (type: int), _col1 (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int), _col1 (type: int)
+                        key expressions: 1 (type: int), _col1 (type: int)
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                        Map-reduce partition columns: 1 (type: int), _col1 (type: int)
                         Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           TableScan
             alias: c
@@ -54,32 +54,32 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: foo (type: int)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
-                    expressions: 1 (type: int), _col1 (type: int)
-                    outputColumnNames: _col0, _col1
+                    expressions: _col0 (type: int)
+                    outputColumnNames: _col1
                     Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int), _col1 (type: int)
+                      keys: 1 (type: int), _col1 (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int), _col1 (type: int)
+                        key expressions: 1 (type: int), _col1 (type: int)
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                        Map-reduce partition columns: 1 (type: int), _col1 (type: int)
                         Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
-          keys: KEY._col0 (type: int), KEY._col1 (type: int)
+          keys: 1 (type: int), KEY._col1 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
-            expressions: _col1 (type: int), _col0 (type: int)
+            expressions: _col1 (type: int), 1 (type: int)
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/perf/query66.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query66.q.out b/ql/src/test/results/clientpositive/perf/query66.q.out
index d698602..a606946 100644
--- a/ql/src/test/results/clientpositive/perf/query66.q.out
+++ b/ql/src/test/results/clientpositive/perf/query66.q.out
@@ -464,167 +464,169 @@ Stage-0
             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","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"]
           <-Reducer 8 [SIMPLE_EDGE]
             SHUFFLE [RS_73]
-              Group By Operator [GBY_71] (rows=26136 width=471)
-                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","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)","sum(VALUE._col24)","sum(VALUE._col25)","sum(VALUE._col26)","sum(VALUE._col27)","sum(VALUE._col28)","s
 um(VALUE._col29)","sum(VALUE._col30)","sum(VALUE._col31)","sum(VALUE._col32)","sum(VALUE._col33)","sum(VALUE._col34)","sum(VALUE._col35)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7
-              <-Union 7 [SIMPLE_EDGE]
-                <-Reducer 19 [CONTAINS]
-                  Reduce Output Operator [RS_70]
-                    PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                    Group By Operator [GBY_69] (rows=52272 width=471)
-                      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","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)","sum(_col42)","sum(_col43)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                      Select Operator [SEL_67] (rows=52272 width=471)
-                        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","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"]
-                        Select Operator [SEL_65] (rows=26136 width=471)
-                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31"]
-                          Group By Operator [GBY_64] (rows=26136 width=471)
-                            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","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 2002
-                          <-Reducer 18 [SIMPLE_EDGE]
-                            SHUFFLE [RS_63]
-                              PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 2002
-                              Group By Operator [GBY_62] (rows=52272 width=471)
-                                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","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 2002
-                                Select Operator [SEL_60] (rows=52272 width=471)
-                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"]
-                                  Merge Join Operator [MERGEJOIN_122] (rows=52272 width=471)
-                                    Conds:RS_57._col2=RS_58._col0(Inner),Output:["_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                  <-Map 23 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_58]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_47] (rows=1 width=0)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_114] (rows=1 width=0)
-                                          predicate:((sm_carrier) IN ('DIAMOND', 'AIRBORNE') and sm_ship_mode_sk is not null)
-                                          TableScan [TS_45] (rows=1 width=0)
-                                            default@ship_mode,ship_mode,Tbl:PARTIAL,Col:NONE,Output:["sm_ship_mode_sk","sm_carrier"]
-                                  <-Reducer 17 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_57]
-                                      PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_121] (rows=47520 width=471)
-                                        Conds:RS_54._col1=RS_55._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                      <-Map 22 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_55]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_44] (rows=43200 width=471)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_113] (rows=43200 width=471)
-                                              predicate:(t_time BETWEEN 49530 AND 78330 and t_time_sk is not null)
-                                              TableScan [TS_42] (rows=86400 width=471)
-                                                default@time_dim,time_dim,Tbl:COMPLETE,Col:NONE,Output:["t_time_sk","t_time"]
-                                      <-Reducer 16 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_54]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_120] (rows=40176 width=1119)
-                                            Conds:RS_51._col0=RS_52._col0(Inner),Output:["_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                          <-Map 21 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_52]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_41] (rows=36524 width=1119)
-                                                Output:["_col0","_col2"]
-                                                Filter Operator [FIL_112] (rows=36524 width=1119)
-                                                  predicate:((d_year = 2002) and d_date_sk is not null)
-                                                  TableScan [TS_39] (rows=73049 width=1119)
-                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
-                                          <-Reducer 15 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_51]
-                                              PartitionCols:_col0
-                                              Merge Join Operator [MERGEJOIN_119] (rows=29 width=1054)
-                                                Conds:RS_48._col3=RS_49._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13"]
-                                              <-Map 14 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_48]
-                                                  PartitionCols:_col3
-                                                  Select Operator [SEL_35] (rows=1 width=0)
-                                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                    Filter Operator [FIL_110] (rows=1 width=0)
-                                                      predicate:(cs_warehouse_sk is not null and cs_sold_date_sk is not null and cs_sold_time_sk is not null and cs_ship_mode_sk is not null)
-                                                      TableScan [TS_33] (rows=1 width=0)
-                                                        default@catalog_sales,catalog_sales,Tbl:PARTIAL,Col:NONE,Output:["cs_sold_date_sk","cs_sold_time_sk","cs_ship_mode_sk","cs_warehouse_sk","cs_quantity","cs_ext_sales_price","cs_net_paid_inc_ship_tax"]
-                                              <-Map 20 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_49]
-                                                  PartitionCols:_col0
-                                                  Select Operator [SEL_38] (rows=27 width=1029)
-                                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                    Filter Operator [FIL_111] (rows=27 width=1029)
-                                                      predicate:w_warehouse_sk is not null
-                                                      TableScan [TS_36] (rows=27 width=1029)
-                                                        default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name","w_warehouse_sq_ft","w_city","w_county","w_state","w_country"]
-                <-Reducer 6 [CONTAINS]
-                  Reduce Output Operator [RS_70]
-                    PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                    Group By Operator [GBY_69] (rows=52272 width=471)
-                      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","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)","sum(_col42)","sum(_col43)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                      Select Operator [SEL_67] (rows=52272 width=471)
-                        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","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"]
-                        Select Operator [SEL_32] (rows=26136 width=471)
-                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31"]
-                          Group By Operator [GBY_31] (rows=26136 width=471)
-                            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","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 2002
-                          <-Reducer 5 [SIMPLE_EDGE]
-                            SHUFFLE [RS_30]
-                              PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 2002
-                              Group By Operator [GBY_29] (rows=52272 width=471)
-                                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","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 2002
-                                Select Operator [SEL_27] (rows=52272 width=471)
-                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"]
-                                  Merge Join Operator [MERGEJOIN_118] (rows=52272 width=471)
-                                    Conds:RS_24._col2=RS_25._col0(Inner),Output:["_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                  <-Map 13 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_25]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_14] (rows=1 width=0)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_109] (rows=1 width=0)
-                                          predicate:((sm_carrier) IN ('DIAMOND', 'AIRBORNE') and sm_ship_mode_sk is not null)
-                                          TableScan [TS_12] (rows=1 width=0)
-                                            default@ship_mode,ship_mode,Tbl:PARTIAL,Col:NONE,Output:["sm_ship_mode_sk","sm_carrier"]
-                                  <-Reducer 4 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_24]
-                                      PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_117] (rows=47520 width=471)
-                                        Conds:RS_21._col1=RS_22._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                      <-Map 12 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_22]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_11] (rows=43200 width=471)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_108] (rows=43200 width=471)
-                                              predicate:(t_time BETWEEN 49530 AND 78330 and t_time_sk is not null)
-                                              TableScan [TS_9] (rows=86400 width=471)
-                                                default@time_dim,time_dim,Tbl:COMPLETE,Col:NONE,Output:["t_time_sk","t_time"]
-                                      <-Reducer 3 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_21]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_116] (rows=40176 width=1119)
-                                            Conds:RS_18._col0=RS_19._col0(Inner),Output:["_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                          <-Map 11 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_19]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_8] (rows=36524 width=1119)
-                                                Output:["_col0","_col2"]
-                                                Filter Operator [FIL_107] (rows=36524 width=1119)
-                                                  predicate:((d_year = 2002) and d_date_sk is not null)
-                                                  TableScan [TS_6] (rows=73049 width=1119)
-                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
-                                          <-Reducer 2 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_18]
-                                              PartitionCols:_col0
-                                              Merge Join Operator [MERGEJOIN_115] (rows=29 width=1054)
-                                                Conds:RS_15._col3=RS_16._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13"]
-                                              <-Map 1 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_15]
-                                                  PartitionCols:_col3
-                                                  Select Operator [SEL_2] (rows=1 width=0)
-                                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                    Filter Operator [FIL_105] (rows=1 width=0)
-                                                      predicate:(ws_warehouse_sk is not null and ws_sold_date_sk is not null and ws_sold_time_sk is not null and ws_ship_mode_sk is not null)
-                                                      TableScan [TS_0] (rows=1 width=0)
-                                                        default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_sold_time_sk","ws_ship_mode_sk","ws_warehouse_sk","ws_quantity","ws_sales_price","ws_net_paid_inc_tax"]
-                                              <-Map 10 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_16]
-                                                  PartitionCols:_col0
-                                                  Select Operator [SEL_5] (rows=27 width=1029)
-                                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                    Filter Operator [FIL_106] (rows=27 width=1029)
-                                                      predicate:w_warehouse_sk is not null
-                                                      TableScan [TS_3] (rows=27 width=1029)
-                                                        default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name","w_warehouse_sq_ft","w_city","w_county","w_state","w_country"]
+              Select Operator [SEL_72] (rows=26136 width=471)
+                Output:["_col0","_col1","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col2","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col3","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col4","_col40","_col41","_col42","_col43","_col5","_col8","_col9"]
+                Group By Operator [GBY_71] (rows=26136 width=471)
+                  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","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)","sum(VALUE._col24)","sum(VALUE._col25)","sum(VALUE._col26)","sum(VALUE._col27)","sum(VALUE._col28)",
 "sum(VALUE._col29)","sum(VALUE._col30)","sum(VALUE._col31)","sum(VALUE._col32)","sum(VALUE._col33)","sum(VALUE._col34)","sum(VALUE._col35)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 'DIAMOND,AIRBORNE', 2002
+                <-Union 7 [SIMPLE_EDGE]
+                  <-Reducer 19 [CONTAINS]
+                    Reduce Output Operator [RS_70]
+                      PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 'DIAMOND,AIRBORNE', 2002
+                      Group By Operator [GBY_69] (rows=52272 width=471)
+                        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","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)","sum(_col42)","sum(_col43)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 'DIAMOND,AIRBO
 RNE', 2002
+                        Select Operator [SEL_67] (rows=52272 width=471)
+                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"]
+                          Select Operator [SEL_65] (rows=26136 width=471)
+                            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","_col24","_col25","_col26","_col27","_col28","_col29"]
+                            Group By Operator [GBY_64] (rows=26136 width=471)
+                              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","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 2002
+                            <-Reducer 18 [SIMPLE_EDGE]
+                              SHUFFLE [RS_63]
+                                PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 2002
+                                Group By Operator [GBY_62] (rows=52272 width=471)
+                                  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","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 2002
+                                  Select Operator [SEL_60] (rows=52272 width=471)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"]
+                                    Merge Join Operator [MERGEJOIN_122] (rows=52272 width=471)
+                                      Conds:RS_57._col2=RS_58._col0(Inner),Output:["_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                    <-Map 23 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_58]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_47] (rows=1 width=0)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_114] (rows=1 width=0)
+                                            predicate:((sm_carrier) IN ('DIAMOND', 'AIRBORNE') and sm_ship_mode_sk is not null)
+                                            TableScan [TS_45] (rows=1 width=0)
+                                              default@ship_mode,ship_mode,Tbl:PARTIAL,Col:NONE,Output:["sm_ship_mode_sk","sm_carrier"]
+                                    <-Reducer 17 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_57]
+                                        PartitionCols:_col2
+                                        Merge Join Operator [MERGEJOIN_121] (rows=47520 width=471)
+                                          Conds:RS_54._col1=RS_55._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                        <-Map 22 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_55]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_44] (rows=43200 width=471)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_113] (rows=43200 width=471)
+                                                predicate:(t_time BETWEEN 49530 AND 78330 and t_time_sk is not null)
+                                                TableScan [TS_42] (rows=86400 width=471)
+                                                  default@time_dim,time_dim,Tbl:COMPLETE,Col:NONE,Output:["t_time_sk","t_time"]
+                                        <-Reducer 16 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_54]
+                                            PartitionCols:_col1
+                                            Merge Join Operator [MERGEJOIN_120] (rows=40176 width=1119)
+                                              Conds:RS_51._col0=RS_52._col0(Inner),Output:["_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                            <-Map 21 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_52]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_41] (rows=36524 width=1119)
+                                                  Output:["_col0","_col2"]
+                                                  Filter Operator [FIL_112] (rows=36524 width=1119)
+                                                    predicate:((d_year = 2002) and d_date_sk is not null)
+                                                    TableScan [TS_39] (rows=73049 width=1119)
+                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
+                                            <-Reducer 15 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_51]
+                                                PartitionCols:_col0
+                                                Merge Join Operator [MERGEJOIN_119] (rows=29 width=1054)
+                                                  Conds:RS_48._col3=RS_49._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13"]
+                                                <-Map 14 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_48]
+                                                    PartitionCols:_col3
+                                                    Select Operator [SEL_35] (rows=1 width=0)
+                                                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                      Filter Operator [FIL_110] (rows=1 width=0)
+                                                        predicate:(cs_warehouse_sk is not null and cs_sold_date_sk is not null and cs_sold_time_sk is not null and cs_ship_mode_sk is not null)
+                                                        TableScan [TS_33] (rows=1 width=0)
+                                                          default@catalog_sales,catalog_sales,Tbl:PARTIAL,Col:NONE,Output:["cs_sold_date_sk","cs_sold_time_sk","cs_ship_mode_sk","cs_warehouse_sk","cs_quantity","cs_ext_sales_price","cs_net_paid_inc_ship_tax"]
+                                                <-Map 20 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_49]
+                                                    PartitionCols:_col0
+                                                    Select Operator [SEL_38] (rows=27 width=1029)
+                                                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                      Filter Operator [FIL_111] (rows=27 width=1029)
+                                                        predicate:w_warehouse_sk is not null
+                                                        TableScan [TS_36] (rows=27 width=1029)
+                                                          default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name","w_warehouse_sq_ft","w_city","w_county","w_state","w_country"]
+                  <-Reducer 6 [CONTAINS]
+                    Reduce Output Operator [RS_70]
+                      PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 'DIAMOND,AIRBORNE', 2002
+                      Group By Operator [GBY_69] (rows=52272 width=471)
+                        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","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)","sum(_col42)","sum(_col43)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 'DIAMOND,AIRBO
 RNE', 2002
+                        Select Operator [SEL_67] (rows=52272 width=471)
+                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"]
+                          Select Operator [SEL_32] (rows=26136 width=471)
+                            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","_col24","_col25","_col26","_col27","_col28","_col29"]
+                            Group By Operator [GBY_31] (rows=26136 width=471)
+                              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","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 2002
+                            <-Reducer 5 [SIMPLE_EDGE]
+                              SHUFFLE [RS_30]
+                                PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 2002
+                                Group By Operator [GBY_29] (rows=52272 width=471)
+                                  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","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 2002
+                                  Select Operator [SEL_27] (rows=52272 width=471)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"]
+                                    Merge Join Operator [MERGEJOIN_118] (rows=52272 width=471)
+                                      Conds:RS_24._col2=RS_25._col0(Inner),Output:["_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                    <-Map 13 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_25]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_14] (rows=1 width=0)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_109] (rows=1 width=0)
+                                            predicate:((sm_carrier) IN ('DIAMOND', 'AIRBORNE') and sm_ship_mode_sk is not null)
+                                            TableScan [TS_12] (rows=1 width=0)
+                                              default@ship_mode,ship_mode,Tbl:PARTIAL,Col:NONE,Output:["sm_ship_mode_sk","sm_carrier"]
+                                    <-Reducer 4 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_24]
+                                        PartitionCols:_col2
+                                        Merge Join Operator [MERGEJOIN_117] (rows=47520 width=471)
+                                          Conds:RS_21._col1=RS_22._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                        <-Map 12 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_22]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_11] (rows=43200 width=471)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_108] (rows=43200 width=471)
+                                                predicate:(t_time BETWEEN 49530 AND 78330 and t_time_sk is not null)
+                                                TableScan [TS_9] (rows=86400 width=471)
+                                                  default@time_dim,time_dim,Tbl:COMPLETE,Col:NONE,Output:["t_time_sk","t_time"]
+                                        <-Reducer 3 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_21]
+                                            PartitionCols:_col1
+                                            Merge Join Operator [MERGEJOIN_116] (rows=40176 width=1119)
+                                              Conds:RS_18._col0=RS_19._col0(Inner),Output:["_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                            <-Map 11 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_19]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_8] (rows=36524 width=1119)
+                                                  Output:["_col0","_col2"]
+                                                  Filter Operator [FIL_107] (rows=36524 width=1119)
+                                                    predicate:((d_year = 2002) and d_date_sk is not null)
+                                                    TableScan [TS_6] (rows=73049 width=1119)
+                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
+                                            <-Reducer 2 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_18]
+                                                PartitionCols:_col0
+                                                Merge Join Operator [MERGEJOIN_115] (rows=29 width=1054)
+                                                  Conds:RS_15._col3=RS_16._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13"]
+                                                <-Map 1 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_15]
+                                                    PartitionCols:_col3
+                                                    Select Operator [SEL_2] (rows=1 width=0)
+                                                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                      Filter Operator [FIL_105] (rows=1 width=0)
+                                                        predicate:(ws_warehouse_sk is not null and ws_sold_date_sk is not null and ws_sold_time_sk is not null and ws_ship_mode_sk is not null)
+                                                        TableScan [TS_0] (rows=1 width=0)
+                                                          default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_sold_time_sk","ws_ship_mode_sk","ws_warehouse_sk","ws_quantity","ws_sales_price","ws_net_paid_inc_tax"]
+                                                <-Map 10 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_16]
+                                                    PartitionCols:_col0
+                                                    Select Operator [SEL_5] (rows=27 width=1029)
+                                                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                      Filter Operator [FIL_106] (rows=27 width=1029)
+                                                        predicate:w_warehouse_sk is not null
+                                                        TableScan [TS_3] (rows=27 width=1029)
+                                                          default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name","w_warehouse_sq_ft","w_city","w_county","w_state","w_country"]