You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jd...@apache.org on 2020/02/18 23:47:21 UTC

[hive] branch master updated: HIVE-22815: reduce the unnecessary file system object creation in MROutput (Richard Zhang, reviewed by Gopal Vijayaraghavan)

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

jdere pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new a9ea0ef  HIVE-22815: reduce the unnecessary file system object creation in MROutput (Richard Zhang, reviewed by Gopal Vijayaraghavan)
a9ea0ef is described below

commit a9ea0ef458c348d0271c955dde93be30d623314d
Author: Richard Zhang <ri...@cloudera.com>
AuthorDate: Tue Feb 18 15:44:50 2020 -0800

    HIVE-22815: reduce the unnecessary file system object creation in MROutput (Richard Zhang, reviewed by Gopal Vijayaraghavan)
---
 .../apache/hadoop/hive/ql/exec/tez/DagUtils.java   |  13 +-
 .../hadoop/hive/ql/exec/tez/NullMROutput.java      |  68 ++++++
 ql/src/test/queries/clientpositive/perf/query41.q  |  55 +++++
 .../clientpositive/perf/spark/query41.q.out        | 252 +++++++++++++++++++++
 .../perf/tez/constraints/query41.q.out             | 167 ++++++++++++++
 .../results/clientpositive/perf/tez/query41.q.out  | 167 ++++++++++++++
 6 files changed, 719 insertions(+), 3 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
index 08eec19..19f6886 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
@@ -53,6 +53,7 @@ import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.tez.mapreduce.common.MRInputSplitDistributor;
 import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
+import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.protos.MRRuntimeProtos;
 import org.apache.tez.runtime.library.api.Partitioner;
 import org.apache.tez.runtime.library.cartesianproduct.CartesianProductConfig;
@@ -152,7 +153,7 @@ import org.apache.tez.mapreduce.hadoop.MRInputHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.input.MultiMRInput;
-import org.apache.tez.mapreduce.output.MROutput;
+import org.apache.hadoop.hive.ql.exec.tez.NullMROutput;
 import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
 import org.apache.tez.runtime.library.common.comparator.TezBytesComparator;
@@ -1508,11 +1509,17 @@ public class DagUtils {
       }
     }
 
-
+    final Class outputKlass;
+    if (HiveOutputFormatImpl.class.getName().equals(conf.get("mapred.output.format.class"))) {
+      // Hive uses this output format, when it is going to write all its data through FS operator
+      outputKlass = NullMROutput.class;
+    } else {
+      outputKlass = MROutput.class;
+    }
     // final vertices need to have at least one output
     if (!hasChildren) {
       v.addDataSink("out_"+work.getName(), new DataSinkDescriptor(
-          OutputDescriptor.create(MROutput.class.getName())
+          OutputDescriptor.create(outputKlass.getName())
           .setUserPayload(TezUtils.createUserPayloadFromConf(conf)), null, null));
     }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/NullMROutput.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/NullMROutput.java
new file mode 100644
index 0000000..202b1fa
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/NullMROutput.java
@@ -0,0 +1,68 @@
+/*
+ * 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.exec.tez;
+
+import org.apache.tez.mapreduce.output.MROutput;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.OutputContext;
+import org.apache.tez.runtime.library.api.KeyValueWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+
+
+public class NullMROutput extends MROutput {
+
+    private static final Logger LOG = LoggerFactory.getLogger(NullMROutput.class);
+
+    public NullMROutput(OutputContext outputContext, int numPhysicalOutputs) {
+        super(outputContext, numPhysicalOutputs);
+    }
+
+    @Override
+    public List<Event> initialize() throws IOException, InterruptedException {
+        List<Event> events = initializeBase();
+        return events;
+    }
+
+    /**
+     * Get a key value write to write Map Reduce compatible output
+     */
+    @Override
+    public KeyValueWriter getWriter() throws IOException {
+        return new KeyValueWriter() {
+            @SuppressWarnings("unchecked")
+            @Override
+            public void write(Object key, Object value) throws IOException {
+                throw new IOException("NullMROutput is not configured for actual rows");
+            }
+        };
+    }
+
+    /**
+     * Call this in the processor before finishing to ensure outputs that
+     * outputs have been flushed. Must be called before commit.
+     * @throws IOException
+     */
+    @Override
+    public void flush() throws IOException {
+    }
+}
diff --git a/ql/src/test/queries/clientpositive/perf/query41.q b/ql/src/test/queries/clientpositive/perf/query41.q
new file mode 100644
index 0000000..b579550
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query41.q
@@ -0,0 +1,55 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query41.tpl and seed 1581015815
+explain
+select  distinct(i_product_name)
+ from item i1
+ where i_manufact_id between 970 and 970+40 
+   and (select count(*) as item_cnt
+        from item
+        where (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'frosted' or i_color = 'rose') and 
+        (i_units = 'Lb' or i_units = 'Gross') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'chocolate' or i_color = 'black') and
+        (i_units = 'Box' or i_units = 'Dram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'slate' or i_color = 'magenta') and
+        (i_units = 'Carton' or i_units = 'Bundle') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'cornflower' or i_color = 'firebrick') and
+        (i_units = 'Pound' or i_units = 'Oz') and
+        (i_size = 'medium' or i_size = 'large')
+        ))) or
+       (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'almond' or i_color = 'steel') and 
+        (i_units = 'Tsp' or i_units = 'Case') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'purple' or i_color = 'aquamarine') and
+        (i_units = 'Bunch' or i_units = 'Gram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'lavender' or i_color = 'papaya') and
+        (i_units = 'Pallet' or i_units = 'Cup') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'maroon' or i_color = 'cyan') and
+        (i_units = 'Each' or i_units = 'N/A') and
+        (i_size = 'medium' or i_size = 'large')
+        )))) > 0
+ order by i_product_name
+ limit 100;
+
+-- end query 1 in stream 0 using template query41.tpl
+
diff --git a/ql/src/test/results/clientpositive/perf/spark/query41.q.out b/ql/src/test/results/clientpositive/perf/spark/query41.q.out
new file mode 100644
index 0000000..68309d1
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/spark/query41.q.out
@@ -0,0 +1,252 @@
+PREHOOK: query: explain
+select  distinct(i_product_name)
+ from item i1
+ where i_manufact_id between 970 and 970+40 
+   and (select count(*) as item_cnt
+        from item
+        where (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'frosted' or i_color = 'rose') and 
+        (i_units = 'Lb' or i_units = 'Gross') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'chocolate' or i_color = 'black') and
+        (i_units = 'Box' or i_units = 'Dram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'slate' or i_color = 'magenta') and
+        (i_units = 'Carton' or i_units = 'Bundle') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'cornflower' or i_color = 'firebrick') and
+        (i_units = 'Pound' or i_units = 'Oz') and
+        (i_size = 'medium' or i_size = 'large')
+        ))) or
+       (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'almond' or i_color = 'steel') and 
+        (i_units = 'Tsp' or i_units = 'Case') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'purple' or i_color = 'aquamarine') and
+        (i_units = 'Bunch' or i_units = 'Gram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'lavender' or i_color = 'papaya') and
+        (i_units = 'Pallet' or i_units = 'Cup') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'maroon' or i_color = 'cyan') and
+        (i_units = 'Each' or i_units = 'N/A') and
+        (i_size = 'medium' or i_size = 'large')
+        )))) > 0
+ order by i_product_name
+ limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@item
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select  distinct(i_product_name)
+ from item i1
+ where i_manufact_id between 970 and 970+40 
+   and (select count(*) as item_cnt
+        from item
+        where (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'frosted' or i_color = 'rose') and 
+        (i_units = 'Lb' or i_units = 'Gross') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'chocolate' or i_color = 'black') and
+        (i_units = 'Box' or i_units = 'Dram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'slate' or i_color = 'magenta') and
+        (i_units = 'Carton' or i_units = 'Bundle') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'cornflower' or i_color = 'firebrick') and
+        (i_units = 'Pound' or i_units = 'Oz') and
+        (i_size = 'medium' or i_size = 'large')
+        ))) or
+       (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'almond' or i_color = 'steel') and 
+        (i_units = 'Tsp' or i_units = 'Case') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'purple' or i_color = 'aquamarine') and
+        (i_units = 'Bunch' or i_units = 'Gram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'lavender' or i_color = 'papaya') and
+        (i_units = 'Pallet' or i_units = 'Cup') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'maroon' or i_color = 'cyan') and
+        (i_units = 'Each' or i_units = 'N/A') and
+        (i_size = 'medium' or i_size = 'large')
+        )))) > 0
+ order by i_product_name
+ limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@item
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Spark
+      Edges:
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 7), Reducer 6 (PARTITION-LEVEL SORT, 7)
+        Reducer 3 <- Reducer 2 (GROUP, 6)
+        Reducer 4 <- Reducer 3 (SORT, 1)
+        Reducer 6 <- Map 5 (GROUP, 6)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: i1
+                  filterExpr: (i_manufact_id BETWEEN 970 AND 1010 and i_manufact is not null) (type: boolean)
+                  Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (i_manufact_id BETWEEN 970 AND 1010 and i_manufact is not null) (type: boolean)
+                    Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: i_manufact (type: string), i_product_name (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
+            Execution mode: vectorized
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: item
+                  filterExpr: (i_manufact is not null and (((i_category = 'Women') and (i_color) IN ('frosted', 'rose') and (i_units) IN ('Lb', 'Gross') and (i_size) IN ('medium', 'large')) or ((i_category = 'Women') and (i_color) IN ('chocolate', 'black') and (i_units) IN ('Box', 'Dram') and (i_size) IN ('economy', 'petite')) or ((i_category = 'Men') and (i_color) IN ('slate', 'magenta') and (i_units) IN ('Carton', 'Bundle') and (i_size) IN ('N/A', 'small')) or ((i_category = 'Men') and [...]
+                  Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (i_manufact is not null and (((i_category = 'Women') and (i_color) IN ('frosted', 'rose') and (i_units) IN ('Lb', 'Gross') and (i_size) IN ('medium', 'large')) or ((i_category = 'Women') and (i_color) IN ('chocolate', 'black') and (i_units) IN ('Box', 'Dram') and (i_size) IN ('economy', 'petite')) or ((i_category = 'Men') and (i_color) IN ('slate', 'magenta') and (i_units) IN ('Carton', 'Bundle') and (i_size) IN ('N/A', 'small')) or ((i_category = 'Men') an [...]
+                    Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: i_manufact (type: string)
+                      outputColumnNames: i_manufact
+                      Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                      Group By Operator
+                        aggregations: count()
+                        keys: i_manufact (type: string)
+                        minReductionHashAggr: 0.99
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: string)
+                          Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col1 (type: bigint)
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col1
+                Statistics: Num rows: 508200 Data size: 729916518 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  keys: _col1 (type: string)
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 508200 Data size: 729916518 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: string)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: string)
+                    Statistics: Num rows: 508200 Data size: 729916518 Basic stats: COMPLETE Column stats: NONE
+                    TopN Hash Memory Usage: 0.1
+        Reducer 3 
+            Execution mode: vectorized
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 254100 Data size: 364958259 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Statistics: Num rows: 254100 Data size: 364958259 Basic stats: COMPLETE Column stats: NONE
+                  TopN Hash Memory Usage: 0.1
+        Reducer 4 
+            Execution mode: vectorized
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 254100 Data size: 364958259 Basic stats: COMPLETE Column stats: NONE
+                Limit
+                  Number of rows: 100
+                  Statistics: Num rows: 100 Data size: 143600 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 100 Data size: 143600 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
+        Reducer 6 
+            Execution mode: vectorized
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: (_col1 > 0L) (type: boolean)
+                  Statistics: Num rows: 77000 Data size: 110593409 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col0 (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 77000 Data size: 110593409 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 77000 Data size: 110593409 Basic stats: COMPLETE Column stats: NONE
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 100
+      Processor Tree:
+        ListSink
+
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query41.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query41.q.out
new file mode 100644
index 0000000..75b7642
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query41.q.out
@@ -0,0 +1,167 @@
+PREHOOK: query: explain
+select  distinct(i_product_name)
+ from item i1
+ where i_manufact_id between 970 and 970+40 
+   and (select count(*) as item_cnt
+        from item
+        where (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'frosted' or i_color = 'rose') and 
+        (i_units = 'Lb' or i_units = 'Gross') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'chocolate' or i_color = 'black') and
+        (i_units = 'Box' or i_units = 'Dram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'slate' or i_color = 'magenta') and
+        (i_units = 'Carton' or i_units = 'Bundle') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'cornflower' or i_color = 'firebrick') and
+        (i_units = 'Pound' or i_units = 'Oz') and
+        (i_size = 'medium' or i_size = 'large')
+        ))) or
+       (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'almond' or i_color = 'steel') and 
+        (i_units = 'Tsp' or i_units = 'Case') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'purple' or i_color = 'aquamarine') and
+        (i_units = 'Bunch' or i_units = 'Gram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'lavender' or i_color = 'papaya') and
+        (i_units = 'Pallet' or i_units = 'Cup') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'maroon' or i_color = 'cyan') and
+        (i_units = 'Each' or i_units = 'N/A') and
+        (i_size = 'medium' or i_size = 'large')
+        )))) > 0
+ order by i_product_name
+ limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@item
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain
+select  distinct(i_product_name)
+ from item i1
+ where i_manufact_id between 970 and 970+40 
+   and (select count(*) as item_cnt
+        from item
+        where (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'frosted' or i_color = 'rose') and 
+        (i_units = 'Lb' or i_units = 'Gross') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'chocolate' or i_color = 'black') and
+        (i_units = 'Box' or i_units = 'Dram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'slate' or i_color = 'magenta') and
+        (i_units = 'Carton' or i_units = 'Bundle') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'cornflower' or i_color = 'firebrick') and
+        (i_units = 'Pound' or i_units = 'Oz') and
+        (i_size = 'medium' or i_size = 'large')
+        ))) or
+       (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'almond' or i_color = 'steel') and 
+        (i_units = 'Tsp' or i_units = 'Case') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'purple' or i_color = 'aquamarine') and
+        (i_units = 'Bunch' or i_units = 'Gram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'lavender' or i_color = 'papaya') and
+        (i_units = 'Pallet' or i_units = 'Cup') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'maroon' or i_color = 'cyan') and
+        (i_units = 'Each' or i_units = 'N/A') and
+        (i_size = 'medium' or i_size = 'large')
+        )))) > 0
+ order by i_product_name
+ limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@item
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+Reducer 6 <- Map 5 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 4 vectorized
+      File Output Operator [FS_59]
+        Limit [LIM_58] (rows=100 width=107)
+          Number of rows:100
+          Select Operator [SEL_57] (rows=376 width=107)
+            Output:["_col0"]
+          <-Reducer 3 [SIMPLE_EDGE] vectorized
+            SHUFFLE [RS_56]
+              Group By Operator [GBY_55] (rows=376 width=107)
+                Output:["_col0"],keys:KEY._col0
+              <-Reducer 2 [SIMPLE_EDGE]
+                SHUFFLE [RS_16]
+                  PartitionCols:_col0
+                  Group By Operator [GBY_15] (rows=376 width=107)
+                    Output:["_col0"],keys:_col1
+                    Top N Key Operator [TNK_30] (rows=752 width=107)
+                      keys:_col1,top n:100
+                      Merge Join Operator [MERGEJOIN_43] (rows=752 width=107)
+                        Conds:RS_46._col0=RS_54._col0(Inner),Output:["_col1"]
+                      <-Map 1 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_46]
+                          PartitionCols:_col0
+                          Select Operator [SEL_45] (rows=20726 width=202)
+                            Output:["_col0","_col1"]
+                            Filter Operator [FIL_44] (rows=20726 width=205)
+                              predicate:(i_manufact_id BETWEEN 970 AND 1010 and i_manufact is not null)
+                              TableScan [TS_0] (rows=462000 width=205)
+                                default@item,i1,Tbl:COMPLETE,Col:COMPLETE,Output:["i_manufact_id","i_manufact","i_product_name"]
+                      <-Reducer 6 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_54]
+                          PartitionCols:_col0
+                          Select Operator [SEL_53] (rows=46 width=95)
+                            Output:["_col0"]
+                            Filter Operator [FIL_52] (rows=46 width=103)
+                              predicate:(_col1 > 0L)
+                              Group By Operator [GBY_51] (rows=140 width=103)
+                                Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
+                              <-Map 5 [SIMPLE_EDGE] vectorized
+                                SHUFFLE [RS_50]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_49] (rows=140 width=103)
+                                    Output:["_col0","_col1"],aggregations:["count()"],keys:i_manufact
+                                    Select Operator [SEL_48] (rows=280 width=450)
+                                      Output:["i_manufact"]
+                                      Filter Operator [FIL_47] (rows=280 width=450)
+                                        predicate:((((i_category = 'Women') and (i_color) IN ('frosted', 'rose') and (i_units) IN ('Lb', 'Gross') and (i_size) IN ('medium', 'large')) or ((i_category = 'Women') and (i_color) IN ('chocolate', 'black') and (i_units) IN ('Box', 'Dram') and (i_size) IN ('economy', 'petite')) or ((i_category = 'Men') and (i_color) IN ('slate', 'magenta') and (i_units) IN ('Carton', 'Bundle') and (i_size) IN ('N/A', 'small')) or ((i_category = 'Men') and (i_col [...]
+                                        TableScan [TS_3] (rows=462000 width=450)
+                                          default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_category","i_manufact","i_size","i_color","i_units"]
+
diff --git a/ql/src/test/results/clientpositive/perf/tez/query41.q.out b/ql/src/test/results/clientpositive/perf/tez/query41.q.out
new file mode 100644
index 0000000..75b7642
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/tez/query41.q.out
@@ -0,0 +1,167 @@
+PREHOOK: query: explain
+select  distinct(i_product_name)
+ from item i1
+ where i_manufact_id between 970 and 970+40 
+   and (select count(*) as item_cnt
+        from item
+        where (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'frosted' or i_color = 'rose') and 
+        (i_units = 'Lb' or i_units = 'Gross') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'chocolate' or i_color = 'black') and
+        (i_units = 'Box' or i_units = 'Dram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'slate' or i_color = 'magenta') and
+        (i_units = 'Carton' or i_units = 'Bundle') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'cornflower' or i_color = 'firebrick') and
+        (i_units = 'Pound' or i_units = 'Oz') and
+        (i_size = 'medium' or i_size = 'large')
+        ))) or
+       (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'almond' or i_color = 'steel') and 
+        (i_units = 'Tsp' or i_units = 'Case') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'purple' or i_color = 'aquamarine') and
+        (i_units = 'Bunch' or i_units = 'Gram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'lavender' or i_color = 'papaya') and
+        (i_units = 'Pallet' or i_units = 'Cup') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'maroon' or i_color = 'cyan') and
+        (i_units = 'Each' or i_units = 'N/A') and
+        (i_size = 'medium' or i_size = 'large')
+        )))) > 0
+ order by i_product_name
+ limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@item
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain
+select  distinct(i_product_name)
+ from item i1
+ where i_manufact_id between 970 and 970+40 
+   and (select count(*) as item_cnt
+        from item
+        where (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'frosted' or i_color = 'rose') and 
+        (i_units = 'Lb' or i_units = 'Gross') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'chocolate' or i_color = 'black') and
+        (i_units = 'Box' or i_units = 'Dram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'slate' or i_color = 'magenta') and
+        (i_units = 'Carton' or i_units = 'Bundle') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'cornflower' or i_color = 'firebrick') and
+        (i_units = 'Pound' or i_units = 'Oz') and
+        (i_size = 'medium' or i_size = 'large')
+        ))) or
+       (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'almond' or i_color = 'steel') and 
+        (i_units = 'Tsp' or i_units = 'Case') and
+        (i_size = 'medium' or i_size = 'large')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'purple' or i_color = 'aquamarine') and
+        (i_units = 'Bunch' or i_units = 'Gram') and
+        (i_size = 'economy' or i_size = 'petite')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'lavender' or i_color = 'papaya') and
+        (i_units = 'Pallet' or i_units = 'Cup') and
+        (i_size = 'N/A' or i_size = 'small')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'maroon' or i_color = 'cyan') and
+        (i_units = 'Each' or i_units = 'N/A') and
+        (i_size = 'medium' or i_size = 'large')
+        )))) > 0
+ order by i_product_name
+ limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@item
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+Reducer 6 <- Map 5 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 4 vectorized
+      File Output Operator [FS_59]
+        Limit [LIM_58] (rows=100 width=107)
+          Number of rows:100
+          Select Operator [SEL_57] (rows=376 width=107)
+            Output:["_col0"]
+          <-Reducer 3 [SIMPLE_EDGE] vectorized
+            SHUFFLE [RS_56]
+              Group By Operator [GBY_55] (rows=376 width=107)
+                Output:["_col0"],keys:KEY._col0
+              <-Reducer 2 [SIMPLE_EDGE]
+                SHUFFLE [RS_16]
+                  PartitionCols:_col0
+                  Group By Operator [GBY_15] (rows=376 width=107)
+                    Output:["_col0"],keys:_col1
+                    Top N Key Operator [TNK_30] (rows=752 width=107)
+                      keys:_col1,top n:100
+                      Merge Join Operator [MERGEJOIN_43] (rows=752 width=107)
+                        Conds:RS_46._col0=RS_54._col0(Inner),Output:["_col1"]
+                      <-Map 1 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_46]
+                          PartitionCols:_col0
+                          Select Operator [SEL_45] (rows=20726 width=202)
+                            Output:["_col0","_col1"]
+                            Filter Operator [FIL_44] (rows=20726 width=205)
+                              predicate:(i_manufact_id BETWEEN 970 AND 1010 and i_manufact is not null)
+                              TableScan [TS_0] (rows=462000 width=205)
+                                default@item,i1,Tbl:COMPLETE,Col:COMPLETE,Output:["i_manufact_id","i_manufact","i_product_name"]
+                      <-Reducer 6 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_54]
+                          PartitionCols:_col0
+                          Select Operator [SEL_53] (rows=46 width=95)
+                            Output:["_col0"]
+                            Filter Operator [FIL_52] (rows=46 width=103)
+                              predicate:(_col1 > 0L)
+                              Group By Operator [GBY_51] (rows=140 width=103)
+                                Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
+                              <-Map 5 [SIMPLE_EDGE] vectorized
+                                SHUFFLE [RS_50]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_49] (rows=140 width=103)
+                                    Output:["_col0","_col1"],aggregations:["count()"],keys:i_manufact
+                                    Select Operator [SEL_48] (rows=280 width=450)
+                                      Output:["i_manufact"]
+                                      Filter Operator [FIL_47] (rows=280 width=450)
+                                        predicate:((((i_category = 'Women') and (i_color) IN ('frosted', 'rose') and (i_units) IN ('Lb', 'Gross') and (i_size) IN ('medium', 'large')) or ((i_category = 'Women') and (i_color) IN ('chocolate', 'black') and (i_units) IN ('Box', 'Dram') and (i_size) IN ('economy', 'petite')) or ((i_category = 'Men') and (i_color) IN ('slate', 'magenta') and (i_units) IN ('Carton', 'Bundle') and (i_size) IN ('N/A', 'small')) or ((i_category = 'Men') and (i_col [...]
+                                        TableScan [TS_3] (rows=462000 width=450)
+                                          default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_category","i_manufact","i_size","i_color","i_units"]
+