You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by we...@apache.org on 2017/05/31 00:11:39 UTC

[01/17] hive git commit: HIVE-16665: Race condition in Utilities.GetInputPathsCallable --> createDummyFileForEmptyPartition (Sahil Takiar, reviewed by Sergio Pena, Vihang Karajgaonkar)

Repository: hive
Updated Branches:
  refs/heads/hive-14535 7ab6f01c1 -> 330215043


HIVE-16665: Race condition in Utilities.GetInputPathsCallable --> createDummyFileForEmptyPartition (Sahil Takiar, reviewed by Sergio Pena, Vihang Karajgaonkar)


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

Branch: refs/heads/hive-14535
Commit: 824b9c80b443dc4e2b9ad35214a23ac756e75234
Parents: 4cd4251
Author: Sahil Takiar <ta...@gmail.com>
Authored: Tue May 30 13:43:32 2017 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Tue May 30 13:43:32 2017 -0500

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/Utilities.java   |  48 +++++----
 .../hadoop/hive/ql/exec/TestUtilities.java      | 104 +++++++++++++++++--
 2 files changed, 122 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/824b9c80/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index ebf1344..c70e1e0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -3110,22 +3110,30 @@ public final class Utilities {
     }
 
     List<Path> finalPathsToAdd = new LinkedList<>();
-    List<Future<Path>> futures = new LinkedList<>();
+    Map<GetInputPathsCallable, Future<Path>> getPathsCallableToFuture = new LinkedHashMap<>();
     for (final Path path : pathsToAdd) {
-      if (lDrvStat != null && lDrvStat.driverState == DriverState.INTERRUPT)
+      if (lDrvStat != null && lDrvStat.driverState == DriverState.INTERRUPT) {
         throw new IOException("Operation is Canceled. ");
+      }
       if (pool == null) {
-        finalPathsToAdd.add(new GetInputPathsCallable(path, job, work, hiveScratchDir, ctx, skipDummy).call());
+        Path newPath = new GetInputPathsCallable(path, job, work, hiveScratchDir, ctx, skipDummy).call();
+        updatePathForMapWork(newPath, work, path);
+        finalPathsToAdd.add(newPath);
       } else {
-        futures.add(pool.submit(new GetInputPathsCallable(path, job, work, hiveScratchDir, ctx, skipDummy)));
+        GetInputPathsCallable callable = new GetInputPathsCallable(path, job, work, hiveScratchDir, ctx, skipDummy);
+        getPathsCallableToFuture.put(callable, pool.submit(callable));
       }
     }
 
     if (pool != null) {
-      for (Future<Path> future : futures) {
-        if (lDrvStat != null && lDrvStat.driverState == DriverState.INTERRUPT)
+      for (Map.Entry<GetInputPathsCallable, Future<Path>> future : getPathsCallableToFuture.entrySet()) {
+        if (lDrvStat != null && lDrvStat.driverState == DriverState.INTERRUPT) {
           throw new IOException("Operation is Canceled. ");
-        finalPathsToAdd.add(future.get());
+        }
+
+        Path newPath = future.getValue().get();
+        updatePathForMapWork(newPath, work, future.getKey().path);
+        finalPathsToAdd.add(newPath);
       }
     }
 
@@ -3154,7 +3162,8 @@ public final class Utilities {
     @Override
     public Path call() throws Exception {
       if (!this.skipDummy && isEmptyPath(this.job, this.path, this.ctx)) {
-        return createDummyFileForEmptyPartition(this.path, this.job, this.work, this.hiveScratchDir);
+        return createDummyFileForEmptyPartition(this.path, this.job, this.work.getPathToPartitionInfo().get(this.path),
+                this.hiveScratchDir);
       }
       return this.path;
     }
@@ -3192,14 +3201,12 @@ public final class Utilities {
   }
 
   @SuppressWarnings("rawtypes")
-  private static Path createDummyFileForEmptyPartition(Path path, JobConf job, MapWork work,
-      Path hiveScratchDir)
-          throws Exception {
+  private static Path createDummyFileForEmptyPartition(Path path, JobConf job, PartitionDesc partDesc,
+                                                       Path hiveScratchDir) throws Exception {
 
     String strPath = path.toString();
 
     // The input file does not exist, replace it by a empty file
-    PartitionDesc partDesc = work.getPathToPartitionInfo().get(path);
     if (partDesc.getTableDesc().isNonNative()) {
       // if this isn't a hive table we can't create an empty file for it.
       return path;
@@ -3216,16 +3223,19 @@ public final class Utilities {
     if (LOG.isInfoEnabled()) {
       LOG.info("Changed input file " + strPath + " to empty file " + newPath + " (" + oneRow + ")");
     }
+    return newPath;
+  }
 
+  private static void updatePathForMapWork(Path newPath, MapWork work, Path path) {
     // update the work
+    if (!newPath.equals(path)) {
+      PartitionDesc partDesc = work.getPathToPartitionInfo().get(path);
+      work.addPathToAlias(newPath, work.getPathToAliases().get(path));
+      work.removePathToAlias(path);
 
-    work.addPathToAlias(newPath, work.getPathToAliases().get(path));
-    work.removePathToAlias(path);
-
-    work.removePathToPartitionInfo(path);
-    work.addPathToPartitionInfo(newPath, partDesc);
-
-    return newPath;
+      work.removePathToPartitionInfo(path);
+      work.addPathToPartitionInfo(newPath, partDesc);
+    }
   }
 
   @SuppressWarnings("rawtypes")

http://git-wip-us.apache.org/repos/asf/hive/blob/824b9c80/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
index 650f169..434e206 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
@@ -20,6 +20,10 @@ package org.apache.hadoop.hive.ql.exec;
 
 import static org.apache.hadoop.hive.ql.exec.Utilities.DEPRECATED_MAPRED_DFSCLIENT_PARALLELISM_MAX;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.apache.hadoop.hive.ql.exec.Utilities.getFileExtension;
 import static org.mockito.Mockito.doReturn;
@@ -32,6 +36,7 @@ import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
@@ -242,16 +247,17 @@ public class TestUtilities {
 
   /**
    * Check that calling {@link Utilities#getInputPaths(JobConf, MapWork, Path, Context, boolean)}
-   * can process two different empty tables without throwing any exceptions.
+   * can process two different tables that both have empty partitions.
    */
   @Test
-  public void testGetInputPathsWithEmptyTables() throws Exception {
+  public void testGetInputPathsWithEmptyPartitions() throws Exception {
     String alias1Name = "alias1";
     String alias2Name = "alias2";
 
     MapWork mapWork1 = new MapWork();
     MapWork mapWork2 = new MapWork();
     JobConf jobConf = new JobConf();
+    Configuration conf = new Configuration();
 
     Path nonExistentPath1 = new Path(UUID.randomUUID().toString());
     Path nonExistentPath2 = new Path(UUID.randomUUID().toString());
@@ -269,14 +275,14 @@ public class TestUtilities {
 
     mapWork1.setPathToAliases(new LinkedHashMap<>(
             ImmutableMap.of(nonExistentPath1, Lists.newArrayList(alias1Name))));
-    mapWork1.setAliasToWork(new LinkedHashMap<String, Operator<? extends OperatorDesc>>(
+    mapWork1.setAliasToWork(new LinkedHashMap<>(
             ImmutableMap.of(alias1Name, (Operator<?>) mock(Operator.class))));
     mapWork1.setPathToPartitionInfo(new LinkedHashMap<>(
             ImmutableMap.of(nonExistentPath1, mockPartitionDesc)));
 
     mapWork2.setPathToAliases(new LinkedHashMap<>(
             ImmutableMap.of(nonExistentPath2, Lists.newArrayList(alias2Name))));
-    mapWork2.setAliasToWork(new LinkedHashMap<String, Operator<? extends OperatorDesc>>(
+    mapWork2.setAliasToWork(new LinkedHashMap<>(
             ImmutableMap.of(alias2Name, (Operator<?>) mock(Operator.class))));
     mapWork2.setPathToPartitionInfo(new LinkedHashMap<>(
             ImmutableMap.of(nonExistentPath2, mockPartitionDesc)));
@@ -284,11 +290,22 @@ public class TestUtilities {
     List<Path> inputPaths = new ArrayList<>();
     try {
       Path scratchDir = new Path(HiveConf.getVar(jobConf, HiveConf.ConfVars.LOCALSCRATCHDIR));
-      inputPaths.addAll(Utilities.getInputPaths(jobConf, mapWork1, scratchDir,
-              mock(Context.class), false));
-      inputPaths.addAll(Utilities.getInputPaths(jobConf, mapWork2, scratchDir,
-              mock(Context.class), false));
-      assertEquals(inputPaths.size(), 2);
+
+      List<Path> inputPaths1 = Utilities.getInputPaths(jobConf, mapWork1, scratchDir,
+              mock(Context.class), false);
+      inputPaths.addAll(inputPaths1);
+      assertEquals(inputPaths1.size(), 1);
+      assertNotEquals(inputPaths1.get(0), nonExistentPath1);
+      assertTrue(inputPaths1.get(0).getFileSystem(conf).exists(inputPaths1.get(0)));
+      assertFalse(nonExistentPath1.getFileSystem(conf).exists(nonExistentPath1));
+
+      List<Path> inputPaths2 = Utilities.getInputPaths(jobConf, mapWork2, scratchDir,
+              mock(Context.class), false);
+      inputPaths.addAll(inputPaths2);
+      assertEquals(inputPaths2.size(), 1);
+      assertNotEquals(inputPaths2.get(0), nonExistentPath2);
+      assertTrue(inputPaths2.get(0).getFileSystem(conf).exists(inputPaths2.get(0)));
+      assertFalse(nonExistentPath2.getFileSystem(conf).exists(nonExistentPath2));
     } finally {
       File file;
       for (Path path : inputPaths) {
@@ -301,7 +318,72 @@ public class TestUtilities {
   }
 
   /**
-   * Check that calling {@link Utilities#getMaxExecutorsForInputListing(JobConf, int)}
+   * Check that calling {@link Utilities#getInputPaths(JobConf, MapWork, Path, Context, boolean)}
+   * can process two different tables that both have empty partitions when using multiple threads.
+   * Some extra logic is placed at the end of the test to validate no race conditions put the
+   * {@link MapWork} object in an invalid state.
+   */
+  @Test
+  public void testGetInputPathsWithMultipleThreadsAndEmptyPartitions() throws Exception {
+    int numPartitions = 15;
+    JobConf jobConf = new JobConf();
+    jobConf.setInt(HiveConf.ConfVars.HIVE_EXEC_INPUT_LISTING_MAX_THREADS.varname,
+            Runtime.getRuntime().availableProcessors() * 2);
+    MapWork mapWork = new MapWork();
+    Path testTablePath = new Path("testTable");
+    Path[] testPartitionsPaths = new Path[numPartitions];
+
+    PartitionDesc mockPartitionDesc = mock(PartitionDesc.class);
+    TableDesc mockTableDesc = mock(TableDesc.class);
+
+    when(mockTableDesc.isNonNative()).thenReturn(false);
+    when(mockTableDesc.getProperties()).thenReturn(new Properties());
+    when(mockPartitionDesc.getProperties()).thenReturn(new Properties());
+    when(mockPartitionDesc.getTableDesc()).thenReturn(mockTableDesc);
+    doReturn(HiveSequenceFileOutputFormat.class).when(
+            mockPartitionDesc).getOutputFileFormatClass();
+
+
+    for (int i = 0; i < numPartitions; i++) {
+      String testPartitionName = "p=" + i;
+      testPartitionsPaths[i] = new Path(testTablePath, "p=" + i);
+      mapWork.getPathToAliases().put(testPartitionsPaths[i], Lists.newArrayList(testPartitionName));
+      mapWork.getAliasToWork().put(testPartitionName, (Operator<?>) mock(Operator.class));
+      mapWork.getPathToPartitionInfo().put(testPartitionsPaths[i], mockPartitionDesc);
+
+    }
+
+    FileSystem fs = FileSystem.getLocal(jobConf);
+
+    try {
+      fs.mkdirs(testTablePath);
+      List<Path> inputPaths = Utilities.getInputPaths(jobConf, mapWork,
+              new Path(HiveConf.getVar(jobConf, HiveConf.ConfVars.LOCALSCRATCHDIR)), mock(Context.class), false);
+      assertEquals(inputPaths.size(), numPartitions);
+
+      for (int i = 0; i < numPartitions; i++) {
+        assertNotEquals(inputPaths.get(i), testPartitionsPaths[i]);
+      }
+
+      assertEquals(mapWork.getPathToAliases().size(), numPartitions);
+      assertEquals(mapWork.getPathToPartitionInfo().size(), numPartitions);
+      assertEquals(mapWork.getAliasToWork().size(), numPartitions);
+
+      for (Map.Entry<Path, ArrayList<String>> entry : mapWork.getPathToAliases().entrySet()) {
+        assertNotNull(entry.getKey());
+        assertNotNull(entry.getValue());
+        assertEquals(entry.getValue().size(), 1);
+        assertTrue(entry.getKey().getFileSystem(new Configuration()).exists(entry.getKey()));
+      }
+    } finally {
+      if (fs.exists(testTablePath)) {
+        fs.delete(testTablePath, true);
+      }
+    }
+  }
+
+  /**
+   * Check that calling {@link Utilities#getMaxExecutorsForInputListing(Configuration, int)}
    * returns the maximum number of executors to use based on the number of input locations.
    */
   @Test
@@ -413,7 +495,7 @@ public class TestUtilities {
     Path testTablePath = new Path(testTableName);
     Path[] testPartitionsPaths = new Path[numOfPartitions];
     for (int i=0; i<numOfPartitions; i++) {
-      String testPartitionName = "p=" + 1;
+      String testPartitionName = "p=" + i;
       testPartitionsPaths[i] = new Path(testTablePath, "p=" + i);
 
       pathToAliasTable.put(testPartitionsPaths[i], Lists.newArrayList(testPartitionName));


[09/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query54.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query54.q.out b/ql/src/test/results/clientpositive/perf/query54.q.out
index e1f5135..3cbcbe3 100644
--- a/ql/src/test/results/clientpositive/perf/query54.q.out
+++ b/ql/src/test/results/clientpositive/perf/query54.q.out
@@ -1,175 +1,396 @@
-PREHOOK: query: explain with my_customers as ( select c_customer_sk , c_current_addr_sk from ( select cs_sold_date_sk sold_date_sk, cs_bill_customer_sk customer_sk, cs_item_sk item_sk from catalog_sales union all select ws_sold_date_sk sold_date_sk, ws_bill_customer_sk customer_sk, ws_item_sk item_sk from web_sales ) cs_or_ws_sales, item, date_dim, customer where sold_date_sk = d_date_sk and item_sk = i_item_sk and i_category = 'Jewelry' and i_class = 'football' and c_customer_sk = cs_or_ws_sales.customer_sk and d_moy = 3 and d_year = 2000 group by c_customer_sk , c_current_addr_sk ) , my_revenue as ( select c_customer_sk, sum(ss_ext_sales_price) as revenue from my_customers, store_sales, customer_address, store, date_dim where c_current_addr_sk = ca_address_sk and ca_county = s_county and ca_state = s_state and ss_sold_date_sk = d_date_sk and c_customer_sk = ss_customer_sk and d_month_seq between (1203) and (1205) group by c_customer_sk ) , segments as (select cast((revenue/50) as 
 int) as segment from my_revenue ) select segment, count(*) as num_customers, segment*50 as segment_base from segments group by segment order by segment, num_customers limit 100
+Warning: Shuffle Join MERGEJOIN[191][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[188][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_0, $hdt$_4]] in Stage 'Reducer 12' is a cross product
+Warning: Shuffle Join MERGEJOIN[190][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_0, $hdt$_4, $hdt$_5, $hdt$_6]] in Stage 'Reducer 14' is a cross product
+Warning: Shuffle Join MERGEJOIN[192][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 4' is a cross product
+PREHOOK: query: explain
+with my_customers as (
+ select distinct c_customer_sk
+        , c_current_addr_sk
+ from   
+        ( select cs_sold_date_sk sold_date_sk,
+                 cs_bill_customer_sk customer_sk,
+                 cs_item_sk item_sk
+          from   catalog_sales
+          union all
+          select ws_sold_date_sk sold_date_sk,
+                 ws_bill_customer_sk customer_sk,
+                 ws_item_sk item_sk
+          from   web_sales
+         ) cs_or_ws_sales,
+         item,
+         date_dim,
+         customer
+ where   sold_date_sk = d_date_sk
+         and item_sk = i_item_sk
+         and i_category = 'Jewelry'
+         and i_class = 'consignment'
+         and c_customer_sk = cs_or_ws_sales.customer_sk
+         and d_moy = 3
+         and d_year = 1999
+ )
+ , my_revenue as (
+ select c_customer_sk,
+        sum(ss_ext_sales_price) as revenue
+ from   my_customers,
+        store_sales,
+        customer_address,
+        store,
+        date_dim
+ where  c_current_addr_sk = ca_address_sk
+        and ca_county = s_county
+        and ca_state = s_state
+        and ss_sold_date_sk = d_date_sk
+        and c_customer_sk = ss_customer_sk
+        and d_month_seq between (select distinct d_month_seq+1
+                                 from   date_dim where d_year = 1999 and d_moy = 3)
+                           and  (select distinct d_month_seq+3
+                                 from   date_dim where d_year = 1999 and d_moy = 3)
+ group by c_customer_sk
+ )
+ , segments as
+ (select cast((revenue/50) as int) as segment
+  from   my_revenue
+ )
+  select  segment, count(*) as num_customers, segment*50 as segment_base
+ from segments
+ group by segment
+ order by segment, num_customers
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with my_customers as ( select c_customer_sk , c_current_addr_sk from ( select cs_sold_date_sk sold_date_sk, cs_bill_customer_sk customer_sk, cs_item_sk item_sk from catalog_sales union all select ws_sold_date_sk sold_date_sk, ws_bill_customer_sk customer_sk, ws_item_sk item_sk from web_sales ) cs_or_ws_sales, item, date_dim, customer where sold_date_sk = d_date_sk and item_sk = i_item_sk and i_category = 'Jewelry' and i_class = 'football' and c_customer_sk = cs_or_ws_sales.customer_sk and d_moy = 3 and d_year = 2000 group by c_customer_sk , c_current_addr_sk ) , my_revenue as ( select c_customer_sk, sum(ss_ext_sales_price) as revenue from my_customers, store_sales, customer_address, store, date_dim where c_current_addr_sk = ca_address_sk and ca_county = s_county and ca_state = s_state and ss_sold_date_sk = d_date_sk and c_customer_sk = ss_customer_sk and d_month_seq between (1203) and (1205) group by c_customer_sk ) , segments as (select cast((revenue/50) as
  int) as segment from my_revenue ) select segment, count(*) as num_customers, segment*50 as segment_base from segments group by segment order by segment, num_customers limit 100
+POSTHOOK: query: explain
+with my_customers as (
+ select distinct c_customer_sk
+        , c_current_addr_sk
+ from   
+        ( select cs_sold_date_sk sold_date_sk,
+                 cs_bill_customer_sk customer_sk,
+                 cs_item_sk item_sk
+          from   catalog_sales
+          union all
+          select ws_sold_date_sk sold_date_sk,
+                 ws_bill_customer_sk customer_sk,
+                 ws_item_sk item_sk
+          from   web_sales
+         ) cs_or_ws_sales,
+         item,
+         date_dim,
+         customer
+ where   sold_date_sk = d_date_sk
+         and item_sk = i_item_sk
+         and i_category = 'Jewelry'
+         and i_class = 'consignment'
+         and c_customer_sk = cs_or_ws_sales.customer_sk
+         and d_moy = 3
+         and d_year = 1999
+ )
+ , my_revenue as (
+ select c_customer_sk,
+        sum(ss_ext_sales_price) as revenue
+ from   my_customers,
+        store_sales,
+        customer_address,
+        store,
+        date_dim
+ where  c_current_addr_sk = ca_address_sk
+        and ca_county = s_county
+        and ca_state = s_state
+        and ss_sold_date_sk = d_date_sk
+        and c_customer_sk = ss_customer_sk
+        and d_month_seq between (select distinct d_month_seq+1
+                                 from   date_dim where d_year = 1999 and d_moy = 3)
+                           and  (select distinct d_month_seq+3
+                                 from   date_dim where d_year = 1999 and d_moy = 3)
+ group by c_customer_sk
+ )
+ , segments as
+ (select cast((revenue/50) as int) as segment
+  from   my_revenue
+ )
+  select  segment, count(*) as num_customers, segment*50 as segment_base
+ from segments
+ group by segment
+ order by segment, num_customers
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 12 <- Union 13 (CONTAINS)
-Map 18 <- Union 13 (CONTAINS)
-Reducer 10 <- Reducer 17 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 14 <- Map 19 (SIMPLE_EDGE), Union 13 (SIMPLE_EDGE)
-Reducer 15 <- Map 20 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE)
-Reducer 16 <- Map 21 (SIMPLE_EDGE), Reducer 15 (SIMPLE_EDGE)
-Reducer 17 <- Reducer 16 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 10 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+Map 21 <- Union 22 (CONTAINS)
+Map 27 <- Union 22 (CONTAINS)
+Reducer 10 <- Map 1 (SIMPLE_EDGE)
+Reducer 11 <- Reducer 10 (CUSTOM_SIMPLE_EDGE)
+Reducer 12 <- Reducer 11 (CUSTOM_SIMPLE_EDGE), Reducer 19 (CUSTOM_SIMPLE_EDGE)
+Reducer 13 <- Map 32 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE)
+Reducer 14 <- Reducer 13 (CUSTOM_SIMPLE_EDGE), Reducer 15 (CUSTOM_SIMPLE_EDGE)
+Reducer 15 <- Map 1 (SIMPLE_EDGE)
+Reducer 17 <- Map 16 (SIMPLE_EDGE), Map 20 (SIMPLE_EDGE)
+Reducer 18 <- Reducer 17 (SIMPLE_EDGE), Reducer 26 (SIMPLE_EDGE)
+Reducer 19 <- Map 31 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 23 <- Map 28 (SIMPLE_EDGE), Union 22 (SIMPLE_EDGE)
+Reducer 24 <- Map 29 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE)
+Reducer 25 <- Map 30 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
+Reducer 26 <- Reducer 25 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 9 (CUSTOM_SIMPLE_EDGE)
+Reducer 4 <- Reducer 14 (CUSTOM_SIMPLE_EDGE), Reducer 3 (CUSTOM_SIMPLE_EDGE)
 Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
 Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
-Reducer 9 <- Map 11 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+Reducer 8 <- Map 1 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 8 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:100
     Stage-1
-      Reducer 6
-      File Output Operator [FS_69]
-        Limit [LIM_68] (rows=100 width=88)
+      Reducer 7
+      File Output Operator [FS_129]
+        Limit [LIM_128] (rows=100 width=158)
           Number of rows:100
-          Select Operator [SEL_67] (rows=174238687 width=88)
+          Select Operator [SEL_127] (rows=1614130953450400 width=158)
             Output:["_col0","_col1","_col2"]
-          <-Reducer 5 [SIMPLE_EDGE]
-            SHUFFLE [RS_66]
-              Select Operator [SEL_65] (rows=174238687 width=88)
+          <-Reducer 6 [SIMPLE_EDGE]
+            SHUFFLE [RS_126]
+              Select Operator [SEL_125] (rows=1614130953450400 width=158)
                 Output:["_col0","_col1","_col2"]
-                Group By Operator [GBY_64] (rows=174238687 width=88)
+                Group By Operator [GBY_124] (rows=1614130953450400 width=158)
                   Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
-                <-Reducer 4 [SIMPLE_EDGE]
-                  SHUFFLE [RS_63]
+                <-Reducer 5 [SIMPLE_EDGE]
+                  SHUFFLE [RS_123]
                     PartitionCols:_col0
-                    Group By Operator [GBY_62] (rows=348477374 width=88)
+                    Group By Operator [GBY_122] (rows=3228261906900801 width=158)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                      Select Operator [SEL_60] (rows=348477374 width=88)
+                      Select Operator [SEL_120] (rows=3228261906900801 width=158)
                         Output:["_col0"]
-                        Group By Operator [GBY_59] (rows=348477374 width=88)
+                        Group By Operator [GBY_119] (rows=3228261906900801 width=158)
                           Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                        <-Reducer 3 [SIMPLE_EDGE]
-                          SHUFFLE [RS_58]
+                        <-Reducer 4 [SIMPLE_EDGE]
+                          SHUFFLE [RS_118]
                             PartitionCols:_col0
-                            Group By Operator [GBY_57] (rows=696954748 width=88)
-                              Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col10
-                              Merge Join Operator [MERGEJOIN_117] (rows=696954748 width=88)
-                                Conds:RS_53._col1=RS_54._col5(Inner),Output:["_col2","_col10"]
-                              <-Reducer 10 [SIMPLE_EDGE]
-                                SHUFFLE [RS_54]
-                                  PartitionCols:_col5
-                                  Merge Join Operator [MERGEJOIN_116] (rows=316240138 width=135)
-                                    Conds:RS_46._col0=RS_47._col1(Inner),Output:["_col5"]
-                                  <-Reducer 17 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_47]
-                                      PartitionCols:_col1
-                                      Select Operator [SEL_42] (rows=287491029 width=135)
-                                        Output:["_col0","_col1"]
-                                        Group By Operator [GBY_41] (rows=287491029 width=135)
-                                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                        <-Reducer 16 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_40]
-                                            PartitionCols:_col0, _col1
-                                            Group By Operator [GBY_39] (rows=574982058 width=135)
-                                              Output:["_col0","_col1"],keys:_col10, _col9
-                                              Merge Join Operator [MERGEJOIN_115] (rows=574982058 width=135)
-                                                Conds:RS_35._col1=RS_36._col0(Inner),Output:["_col9","_col10"]
-                                              <-Map 21 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_36]
+                            Group By Operator [GBY_117] (rows=6456523813801603 width=158)
+                              Output:["_col0","_col1"],aggregations:["sum(_col4)"],keys:_col0
+                              Select Operator [SEL_116] (rows=6456523813801603 width=158)
+                                Output:["_col0","_col4"]
+                                Filter Operator [FIL_115] (rows=6456523813801603 width=158)
+                                  predicate:_col11 BETWEEN _col13 AND _col15
+                                  Select Operator [SEL_114] (rows=58108714324214428 width=158)
+                                    Output:["_col0","_col4","_col11","_col13","_col15"]
+                                    Merge Join Operator [MERGEJOIN_192] (rows=58108714324214428 width=158)
+                                      Conds:(Inner),Output:["_col0","_col2","_col6","_col13","_col15"]
+                                    <-Reducer 14 [CUSTOM_SIMPLE_EDGE]
+                                      PARTITION_ONLY_SHUFFLE [RS_112]
+                                        Select Operator [SEL_107] (rows=6363893803988 width=1217)
+                                          Output:["_col0","_col4","_col11","_col13"]
+                                          Merge Join Operator [MERGEJOIN_190] (rows=6363893803988 width=1217)
+                                            Conds:(Left Outer),Output:["_col5","_col9","_col12","_col13"]
+                                          <-Reducer 13 [CUSTOM_SIMPLE_EDGE]
+                                            PARTITION_ONLY_SHUFFLE [RS_104]
+                                              Merge Join Operator [MERGEJOIN_189] (rows=696954748 width=97)
+                                                Conds:RS_101._col7=RS_102._col0(Inner),Output:["_col5","_col9","_col12"]
+                                              <-Map 32 [SIMPLE_EDGE]
+                                                SHUFFLE [RS_102]
                                                   PartitionCols:_col0
-                                                  Select Operator [SEL_28] (rows=80000000 width=860)
+                                                  Select Operator [SEL_80] (rows=73049 width=1119)
                                                     Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_110] (rows=80000000 width=860)
-                                                      predicate:(c_customer_sk is not null and c_current_addr_sk is not null)
-                                                      TableScan [TS_26] (rows=80000000 width=860)
-                                                        default@customer,customer,Tbl:COMPLETE,Col:NONE,Output:["c_customer_sk","c_current_addr_sk"]
-                                              <-Reducer 15 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_35]
-                                                  PartitionCols:_col1
-                                                  Merge Join Operator [MERGEJOIN_114] (rows=522710951 width=135)
-                                                    Conds:RS_32._col2=RS_33._col0(Inner),Output:["_col1"]
-                                                  <-Map 20 [SIMPLE_EDGE]
-                                                    SHUFFLE [RS_33]
-                                                      PartitionCols:_col0
-                                                      Select Operator [SEL_25] (rows=115500 width=1436)
-                                                        Output:["_col0"]
-                                                        Filter Operator [FIL_109] (rows=115500 width=1436)
-                                                          predicate:((i_category = 'Jewelry') and (i_class = 'football') and i_item_sk is not null)
-                                                          TableScan [TS_23] (rows=462000 width=1436)
-                                                            default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_class","i_category"]
-                                                  <-Reducer 14 [SIMPLE_EDGE]
-                                                    SHUFFLE [RS_32]
-                                                      PartitionCols:_col2
-                                                      Merge Join Operator [MERGEJOIN_113] (rows=475191764 width=135)
-                                                        Conds:Union 13._col0=RS_30._col0(Inner),Output:["_col1","_col2"]
-                                                      <-Map 19 [SIMPLE_EDGE]
-                                                        SHUFFLE [RS_30]
-                                                          PartitionCols:_col0
-                                                          Select Operator [SEL_22] (rows=18262 width=1119)
-                                                            Output:["_col0"]
-                                                            Filter Operator [FIL_108] (rows=18262 width=1119)
-                                                              predicate:((d_moy = 3) and (d_year = 2000) and d_date_sk is not null)
-                                                              TableScan [TS_20] (rows=73049 width=1119)
-                                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
-                                                      <-Union 13 [SIMPLE_EDGE]
-                                                        <-Map 12 [CONTAINS]
-                                                          Reduce Output Operator [RS_29]
-                                                            PartitionCols:_col0
-                                                            Select Operator [SEL_14] (rows=287989836 width=135)
-                                                              Output:["_col0","_col1","_col2"]
-                                                              Filter Operator [FIL_106] (rows=287989836 width=135)
-                                                                predicate:(cs_item_sk is not null and cs_sold_date_sk is not null and cs_bill_customer_sk is not null)
-                                                                TableScan [TS_12] (rows=287989836 width=135)
-                                                                  Output:["cs_sold_date_sk","cs_bill_customer_sk","cs_item_sk"]
-                                                        <-Map 18 [CONTAINS]
-                                                          Reduce Output Operator [RS_29]
+                                                    Filter Operator [FIL_180] (rows=73049 width=1119)
+                                                      predicate:d_date_sk is not null
+                                                      TableScan [TS_78] (rows=73049 width=1119)
+                                                        default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_month_seq"]
+                                              <-Reducer 12 [SIMPLE_EDGE]
+                                                SHUFFLE [RS_101]
+                                                  PartitionCols:_col7
+                                                  Merge Join Operator [MERGEJOIN_188] (rows=633595212 width=97)
+                                                    Conds:(Inner),Output:["_col5","_col7","_col9"]
+                                                  <-Reducer 11 [CUSTOM_SIMPLE_EDGE]
+                                                    PARTITION_ONLY_SHUFFLE [RS_99]
+                                                      Select Operator [SEL_77] (rows=1 width=8)
+                                                        Filter Operator [FIL_76] (rows=1 width=8)
+                                                          predicate:(sq_count_check(_col0) <= 1)
+                                                          Group By Operator [GBY_74] (rows=1 width=8)
+                                                            Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                                                          <-Reducer 10 [CUSTOM_SIMPLE_EDGE]
+                                                            PARTITION_ONLY_SHUFFLE [RS_73]
+                                                              Group By Operator [GBY_72] (rows=1 width=8)
+                                                                Output:["_col0"],aggregations:["count()"]
+                                                                Select Operator [SEL_70] (rows=9131 width=1119)
+                                                                  Group By Operator [GBY_69] (rows=9131 width=1119)
+                                                                    Output:["_col0"],keys:KEY._col0
+                                                                  <-Map 1 [SIMPLE_EDGE]
+                                                                    SHUFFLE [RS_68]
+                                                                      PartitionCols:_col0
+                                                                      Group By Operator [GBY_67] (rows=18262 width=1119)
+                                                                        Output:["_col0"],keys:_col0
+                                                                        Select Operator [SEL_65] (rows=18262 width=1119)
+                                                                          Output:["_col0"]
+                                                                          Filter Operator [FIL_179] (rows=18262 width=1119)
+                                                                            predicate:((d_year = 1999) and (d_moy = 3))
+                                                                            TableScan [TS_0] (rows=73049 width=1119)
+                                                                              default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_month_seq","d_year","d_moy"]
+                                                  <-Reducer 19 [CUSTOM_SIMPLE_EDGE]
+                                                    PARTITION_ONLY_SHUFFLE [RS_98]
+                                                      Merge Join Operator [MERGEJOIN_187] (rows=633595212 width=88)
+                                                        Conds:RS_95._col5=RS_96._col1(Inner),Output:["_col5","_col7","_col9"]
+                                                      <-Map 31 [SIMPLE_EDGE]
+                                                        SHUFFLE [RS_96]
+                                                          PartitionCols:_col1
+                                                          Select Operator [SEL_62] (rows=575995635 width=88)
+                                                            Output:["_col0","_col1","_col2"]
+                                                            Filter Operator [FIL_178] (rows=575995635 width=88)
+                                                              predicate:(ss_customer_sk is not null and ss_sold_date_sk is not null)
+                                                              TableScan [TS_60] (rows=575995635 width=88)
+                                                                default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_customer_sk","ss_ext_sales_price"]
+                                                      <-Reducer 18 [SIMPLE_EDGE]
+                                                        SHUFFLE [RS_95]
+                                                          PartitionCols:_col5
+                                                          Merge Join Operator [MERGEJOIN_186] (rows=316240138 width=135)
+                                                            Conds:RS_92._col0=RS_93._col1(Inner),Output:["_col5"]
+                                                          <-Reducer 17 [SIMPLE_EDGE]
+                                                            SHUFFLE [RS_92]
+                                                              PartitionCols:_col0
+                                                              Merge Join Operator [MERGEJOIN_182] (rows=44000000 width=1014)
+                                                                Conds:RS_89._col1, _col2=RS_90._col0, _col1(Inner),Output:["_col0"]
+                                                              <-Map 16 [SIMPLE_EDGE]
+                                                                SHUFFLE [RS_89]
+                                                                  PartitionCols:_col1, _col2
+                                                                  Select Operator [SEL_25] (rows=40000000 width=1014)
+                                                                    Output:["_col0","_col1","_col2"]
+                                                                    Filter Operator [FIL_171] (rows=40000000 width=1014)
+                                                                      predicate:(ca_address_sk is not null and ca_county is not null and ca_state is not null)
+                                                                      TableScan [TS_23] (rows=40000000 width=1014)
+                                                                        default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county","ca_state"]
+                                                              <-Map 20 [SIMPLE_EDGE]
+                                                                SHUFFLE [RS_90]
+                                                                  PartitionCols:_col0, _col1
+                                                                  Select Operator [SEL_28] (rows=1704 width=1910)
+                                                                    Output:["_col0","_col1"]
+                                                                    Filter Operator [FIL_172] (rows=1704 width=1910)
+                                                                      predicate:(s_county is not null and s_state is not null)
+                                                                      TableScan [TS_26] (rows=1704 width=1910)
+                                                                        default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_county","s_state"]
+                                                          <-Reducer 26 [SIMPLE_EDGE]
+                                                            SHUFFLE [RS_93]
+                                                              PartitionCols:_col1
+                                                              Select Operator [SEL_59] (rows=287491029 width=135)
+                                                                Output:["_col0","_col1"]
+                                                                Group By Operator [GBY_58] (rows=287491029 width=135)
+                                                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                                                <-Reducer 25 [SIMPLE_EDGE]
+                                                                  SHUFFLE [RS_57]
+                                                                    PartitionCols:_col0, _col1
+                                                                    Group By Operator [GBY_56] (rows=574982058 width=135)
+                                                                      Output:["_col0","_col1"],keys:_col10, _col9
+                                                                      Merge Join Operator [MERGEJOIN_185] (rows=574982058 width=135)
+                                                                        Conds:RS_52._col1=RS_53._col0(Inner),Output:["_col9","_col10"]
+                                                                      <-Map 30 [SIMPLE_EDGE]
+                                                                        SHUFFLE [RS_53]
+                                                                          PartitionCols:_col0
+                                                                          Select Operator [SEL_45] (rows=80000000 width=860)
+                                                                            Output:["_col0","_col1"]
+                                                                            Filter Operator [FIL_177] (rows=80000000 width=860)
+                                                                              predicate:(c_customer_sk is not null and c_current_addr_sk is not null)
+                                                                              TableScan [TS_43] (rows=80000000 width=860)
+                                                                                default@customer,customer,Tbl:COMPLETE,Col:NONE,Output:["c_customer_sk","c_current_addr_sk"]
+                                                                      <-Reducer 24 [SIMPLE_EDGE]
+                                                                        SHUFFLE [RS_52]
+                                                                          PartitionCols:_col1
+                                                                          Merge Join Operator [MERGEJOIN_184] (rows=522710951 width=135)
+                                                                            Conds:RS_49._col2=RS_50._col0(Inner),Output:["_col1"]
+                                                                          <-Map 29 [SIMPLE_EDGE]
+                                                                            SHUFFLE [RS_50]
+                                                                              PartitionCols:_col0
+                                                                              Select Operator [SEL_42] (rows=115500 width=1436)
+                                                                                Output:["_col0"]
+                                                                                Filter Operator [FIL_176] (rows=115500 width=1436)
+                                                                                  predicate:((i_category = 'Jewelry') and (i_class = 'consignment') and i_item_sk is not null)
+                                                                                  TableScan [TS_40] (rows=462000 width=1436)
+                                                                                    default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_class","i_category"]
+                                                                          <-Reducer 23 [SIMPLE_EDGE]
+                                                                            SHUFFLE [RS_49]
+                                                                              PartitionCols:_col2
+                                                                              Merge Join Operator [MERGEJOIN_183] (rows=475191764 width=135)
+                                                                                Conds:Union 22._col0=RS_47._col0(Inner),Output:["_col1","_col2"]
+                                                                              <-Map 28 [SIMPLE_EDGE]
+                                                                                SHUFFLE [RS_47]
+                                                                                  PartitionCols:_col0
+                                                                                  Select Operator [SEL_39] (rows=18262 width=1119)
+                                                                                    Output:["_col0"]
+                                                                                    Filter Operator [FIL_175] (rows=18262 width=1119)
+                                                                                      predicate:((d_moy = 3) and (d_year = 1999) and d_date_sk is not null)
+                                                                                      TableScan [TS_37] (rows=73049 width=1119)
+                                                                                        default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
+                                                                              <-Union 22 [SIMPLE_EDGE]
+                                                                                <-Map 21 [CONTAINS]
+                                                                                  Reduce Output Operator [RS_46]
+                                                                                    PartitionCols:_col0
+                                                                                    Select Operator [SEL_31] (rows=287989836 width=135)
+                                                                                      Output:["_col0","_col1","_col2"]
+                                                                                      Filter Operator [FIL_173] (rows=287989836 width=135)
+                                                                                        predicate:(cs_item_sk is not null and cs_sold_date_sk is not null and cs_bill_customer_sk is not null)
+                                                                                        TableScan [TS_29] (rows=287989836 width=135)
+                                                                                          Output:["cs_sold_date_sk","cs_bill_customer_sk","cs_item_sk"]
+                                                                                <-Map 27 [CONTAINS]
+                                                                                  Reduce Output Operator [RS_46]
+                                                                                    PartitionCols:_col0
+                                                                                    Select Operator [SEL_34] (rows=144002668 width=135)
+                                                                                      Output:["_col0","_col1","_col2"]
+                                                                                      Filter Operator [FIL_174] (rows=144002668 width=135)
+                                                                                        predicate:(ws_item_sk is not null and ws_sold_date_sk is not null and ws_bill_customer_sk is not null)
+                                                                                        TableScan [TS_32] (rows=144002668 width=135)
+                                                                                          Output:["ws_sold_date_sk","ws_item_sk","ws_bill_customer_sk"]
+                                          <-Reducer 15 [CUSTOM_SIMPLE_EDGE]
+                                            PARTITION_ONLY_SHUFFLE [RS_105]
+                                              Group By Operator [GBY_87] (rows=9131 width=1119)
+                                                Output:["_col0"],keys:KEY._col0
+                                              <-Map 1 [SIMPLE_EDGE]
+                                                SHUFFLE [RS_86]
+                                                  PartitionCols:_col0
+                                                  Group By Operator [GBY_85] (rows=18262 width=1119)
+                                                    Output:["_col0"],keys:_col0
+                                                    Select Operator [SEL_83] (rows=18262 width=1119)
+                                                      Output:["_col0"]
+                                                      Filter Operator [FIL_181] (rows=18262 width=1119)
+                                                        predicate:((d_year = 1999) and (d_moy = 3))
+                                                         Please refer to the previous TableScan [TS_0]
+                                    <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
+                                      PARTITION_ONLY_SHUFFLE [RS_111]
+                                        Merge Join Operator [MERGEJOIN_191] (rows=9131 width=1128)
+                                          Conds:(Right Outer),Output:["_col0"]
+                                        <-Reducer 2 [CUSTOM_SIMPLE_EDGE]
+                                          PARTITION_ONLY_SHUFFLE [RS_108]
+                                            Group By Operator [GBY_6] (rows=9131 width=1119)
+                                              Output:["_col0"],keys:KEY._col0
+                                            <-Map 1 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_5]
+                                                PartitionCols:_col0
+                                                Group By Operator [GBY_4] (rows=18262 width=1119)
+                                                  Output:["_col0"],keys:_col0
+                                                  Select Operator [SEL_2] (rows=18262 width=1119)
+                                                    Output:["_col0"]
+                                                    Filter Operator [FIL_169] (rows=18262 width=1119)
+                                                      predicate:((d_year = 1999) and (d_moy = 3))
+                                                       Please refer to the previous TableScan [TS_0]
+                                        <-Reducer 9 [CUSTOM_SIMPLE_EDGE]
+                                          PARTITION_ONLY_SHUFFLE [RS_109]
+                                            Select Operator [SEL_22] (rows=1 width=8)
+                                              Filter Operator [FIL_21] (rows=1 width=8)
+                                                predicate:(sq_count_check(_col0) <= 1)
+                                                Group By Operator [GBY_19] (rows=1 width=8)
+                                                  Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                                                <-Reducer 8 [CUSTOM_SIMPLE_EDGE]
+                                                  PARTITION_ONLY_SHUFFLE [RS_18]
+                                                    Group By Operator [GBY_17] (rows=1 width=8)
+                                                      Output:["_col0"],aggregations:["count()"]
+                                                      Select Operator [SEL_15] (rows=9131 width=1119)
+                                                        Group By Operator [GBY_14] (rows=9131 width=1119)
+                                                          Output:["_col0"],keys:KEY._col0
+                                                        <-Map 1 [SIMPLE_EDGE]
+                                                          SHUFFLE [RS_13]
                                                             PartitionCols:_col0
-                                                            Select Operator [SEL_17] (rows=144002668 width=135)
-                                                              Output:["_col0","_col1","_col2"]
-                                                              Filter Operator [FIL_107] (rows=144002668 width=135)
-                                                                predicate:(ws_item_sk is not null and ws_sold_date_sk is not null and ws_bill_customer_sk is not null)
-                                                                TableScan [TS_15] (rows=144002668 width=135)
-                                                                  Output:["ws_sold_date_sk","ws_item_sk","ws_bill_customer_sk"]
-                                  <-Reducer 9 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_46]
-                                      PartitionCols:_col0
-                                      Merge Join Operator [MERGEJOIN_112] (rows=44000000 width=1014)
-                                        Conds:RS_43._col1, _col2=RS_44._col0, _col1(Inner),Output:["_col0"]
-                                      <-Map 11 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_44]
-                                          PartitionCols:_col0, _col1
-                                          Select Operator [SEL_11] (rows=1704 width=1910)
-                                            Output:["_col0","_col1"]
-                                            Filter Operator [FIL_105] (rows=1704 width=1910)
-                                              predicate:(s_county is not null and s_state is not null)
-                                              TableScan [TS_9] (rows=1704 width=1910)
-                                                default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_county","s_state"]
-                                      <-Map 8 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_43]
-                                          PartitionCols:_col1, _col2
-                                          Select Operator [SEL_8] (rows=40000000 width=1014)
-                                            Output:["_col0","_col1","_col2"]
-                                            Filter Operator [FIL_104] (rows=40000000 width=1014)
-                                              predicate:(ca_address_sk is not null and ca_county is not null and ca_state is not null)
-                                              TableScan [TS_6] (rows=40000000 width=1014)
-                                                default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county","ca_state"]
-                              <-Reducer 2 [SIMPLE_EDGE]
-                                SHUFFLE [RS_53]
-                                  PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_111] (rows=633595212 width=88)
-                                    Conds:RS_50._col0=RS_51._col0(Inner),Output:["_col1","_col2"]
-                                  <-Map 1 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_50]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_2] (rows=575995635 width=88)
-                                        Output:["_col0","_col1","_col2"]
-                                        Filter Operator [FIL_102] (rows=575995635 width=88)
-                                          predicate:(ss_customer_sk is not null and ss_sold_date_sk is not null)
-                                          TableScan [TS_0] (rows=575995635 width=88)
-                                            default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_customer_sk","ss_ext_sales_price"]
-                                  <-Map 7 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_51]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_5] (rows=8116 width=1119)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_103] (rows=8116 width=1119)
-                                          predicate:(d_month_seq BETWEEN 1203 AND 1205 and d_date_sk is not null)
-                                          TableScan [TS_3] (rows=73049 width=1119)
-                                            default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_month_seq"]
+                                                            Group By Operator [GBY_12] (rows=18262 width=1119)
+                                                              Output:["_col0"],keys:_col0
+                                                              Select Operator [SEL_10] (rows=18262 width=1119)
+                                                                Output:["_col0"]
+                                                                Filter Operator [FIL_170] (rows=18262 width=1119)
+                                                                  predicate:((d_year = 1999) and (d_moy = 3))
+                                                                   Please refer to the previous TableScan [TS_0]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query55.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query55.q.out b/ql/src/test/results/clientpositive/perf/query55.q.out
index 7948b6d..7125422 100644
--- a/ql/src/test/results/clientpositive/perf/query55.q.out
+++ b/ql/src/test/results/clientpositive/perf/query55.q.out
@@ -1,6 +1,28 @@
-PREHOOK: query: explain select i_brand_id brand_id, i_brand brand, sum(ss_ext_sales_price) ext_price from date_dim, store_sales, item where date_dim.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and i_manager_id=36 and d_moy=12 and d_year=2001 group by i_brand, i_brand_id order by ext_price desc, i_brand_id limit 100
+PREHOOK: query: explain
+select  i_brand_id brand_id, i_brand brand,
+ 	sum(ss_ext_sales_price) ext_price
+ from date_dim, store_sales, item
+ where d_date_sk = ss_sold_date_sk
+ 	and ss_item_sk = i_item_sk
+ 	and i_manager_id=36
+ 	and d_moy=12
+ 	and d_year=2001
+ group by i_brand, i_brand_id
+ order by ext_price desc, i_brand_id
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select i_brand_id brand_id, i_brand brand, sum(ss_ext_sales_price) ext_price from date_dim, store_sales, item where date_dim.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and i_manager_id=36 and d_moy=12 and d_year=2001 group by i_brand, i_brand_id order by ext_price desc, i_brand_id limit 100
+POSTHOOK: query: explain
+select  i_brand_id brand_id, i_brand brand,
+ 	sum(ss_ext_sales_price) ext_price
+ from date_dim, store_sales, item
+ where d_date_sk = ss_sold_date_sk
+ 	and ss_item_sk = i_item_sk
+ 	and i_manager_id=36
+ 	and d_moy=12
+ 	and d_year=2001
+ group by i_brand, i_brand_id
+ order by ext_price desc, i_brand_id
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query56.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query56.q.out b/ql/src/test/results/clientpositive/perf/query56.q.out
index 9e710e0..4ec7201 100644
--- a/ql/src/test/results/clientpositive/perf/query56.q.out
+++ b/ql/src/test/results/clientpositive/perf/query56.q.out
@@ -1,4 +1,5 @@
-PREHOOK: query: explain with ss as (
+PREHOOK: query: explain
+with ss as (
  select i_item_id,sum(ss_ext_sales_price) total_sales
  from
  	store_sales,
@@ -64,7 +65,8 @@ where i_color in ('orchid','chiffon','lace'))
  order by total_sales
  limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with ss as (
+POSTHOOK: query: explain
+with ss as (
  select i_item_id,sum(ss_ext_sales_price) total_sales
  from
  	store_sales,


[06/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query70.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query70.q.out b/ql/src/test/results/clientpositive/perf/query70.q.out
index d4b7731..ee1fe86 100644
--- a/ql/src/test/results/clientpositive/perf/query70.q.out
+++ b/ql/src/test/results/clientpositive/perf/query70.q.out
@@ -5,25 +5,25 @@ select
    ,s_county
    ,grouping(s_state)+grouping(s_county) as lochierarchy
    ,rank() over (
-     partition by grouping(s_state)+grouping(s_county),
-     case when grouping(s_county) = 0 then s_state end 
-     order by sum(ss_net_profit) desc) as rank_within_parent
+ 	partition by grouping(s_state)+grouping(s_county),
+ 	case when grouping(s_county) = 0 then s_state end 
+ 	order by sum(ss_net_profit) desc) as rank_within_parent
  from
     store_sales
    ,date_dim       d1
-   ,store s
+   ,store
  where
     d1.d_month_seq between 1212 and 1212+11
  and d1.d_date_sk = ss_sold_date_sk
  and s_store_sk  = ss_store_sk
- and s.s_state in
+ and s_state in
              ( select s_state
                from  (select s_state as s_state,
-                 rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking
+ 			    rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking
                       from   store_sales, store, date_dim
                       where  d_month_seq between 1212 and 1212+11
-                 and d_date_sk = ss_sold_date_sk
-                 and s_store_sk  = ss_store_sk
+ 			    and d_date_sk = ss_sold_date_sk
+ 			    and s_store_sk  = ss_store_sk
                       group by s_state
                      ) tmp1 
                where ranking <= 5
@@ -42,25 +42,25 @@ select
    ,s_county
    ,grouping(s_state)+grouping(s_county) as lochierarchy
    ,rank() over (
-     partition by grouping(s_state)+grouping(s_county),
-     case when grouping(s_county) = 0 then s_state end 
-     order by sum(ss_net_profit) desc) as rank_within_parent
+ 	partition by grouping(s_state)+grouping(s_county),
+ 	case when grouping(s_county) = 0 then s_state end 
+ 	order by sum(ss_net_profit) desc) as rank_within_parent
  from
     store_sales
    ,date_dim       d1
-   ,store s
+   ,store
  where
     d1.d_month_seq between 1212 and 1212+11
  and d1.d_date_sk = ss_sold_date_sk
  and s_store_sk  = ss_store_sk
- and s.s_state in
+ and s_state in
              ( select s_state
                from  (select s_state as s_state,
-                 rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking
+ 			    rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking
                       from   store_sales, store, date_dim
                       where  d_month_seq between 1212 and 1212+11
-                 and d_date_sk = ss_sold_date_sk
-                 and s_store_sk  = ss_store_sk
+ 			    and d_date_sk = ss_sold_date_sk
+ 			    and s_store_sk  = ss_store_sk
                       group by s_state
                      ) tmp1 
                where ranking <= 5
@@ -188,7 +188,7 @@ Stage-0
                                           Filter Operator [FIL_80] (rows=1704 width=1910)
                                             predicate:(s_state is not null and s_store_sk is not null)
                                             TableScan [TS_6] (rows=1704 width=1910)
-                                              default@store,s,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_county","s_state"]
+                                              default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_county","s_state"]
                                     <-Reducer 2 [SIMPLE_EDGE]
                                       SHUFFLE [RS_40]
                                         PartitionCols:_col1

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query71.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query71.q.out b/ql/src/test/results/clientpositive/perf/query71.q.out
index 6d85629..4465808 100644
--- a/ql/src/test/results/clientpositive/perf/query71.q.out
+++ b/ql/src/test/results/clientpositive/perf/query71.q.out
@@ -1,6 +1,78 @@
-PREHOOK: query: explain select i_brand_id brand_id, i_brand brand,t_hour,t_minute, sum(ext_price) ext_price from item JOIN (select ws_ext_sales_price as ext_price, ws_sold_date_sk as sold_date_sk, ws_item_sk as sold_item_sk, ws_sold_time_sk as time_sk from web_sales,date_dim where date_dim.d_date_sk = web_sales.ws_sold_date_sk and d_moy=12 and d_year=2001 union all select cs_ext_sales_price as ext_price, cs_sold_date_sk as sold_date_sk, cs_item_sk as sold_item_sk, cs_sold_time_sk as time_sk from catalog_sales,date_dim where date_dim.d_date_sk = catalog_sales.cs_sold_date_sk and d_moy=12 and d_year=2001 union all select ss_ext_sales_price as ext_price, ss_sold_date_sk as sold_date_sk, ss_item_sk as sold_item_sk, ss_sold_time_sk as time_sk from store_sales,date_dim where date_dim.d_date_sk = store_sales.ss_sold_date_sk and d_moy=12 and d_year=2001 ) tmp ON tmp.sold_item_sk = item.i_item_sk JOIN time_dim ON tmp.time_sk = time_dim.t_time_sk where i_manager_id=1 and (t_meal_time = 'break
 fast' or t_meal_time = 'dinner') group by i_brand, i_brand_id,t_hour,t_minute order by ext_price desc, i_brand_id
+PREHOOK: query: explain
+select i_brand_id brand_id, i_brand brand,t_hour,t_minute,
+ 	sum(ext_price) ext_price
+ from item, (select ws_ext_sales_price as ext_price, 
+                        ws_sold_date_sk as sold_date_sk,
+                        ws_item_sk as sold_item_sk,
+                        ws_sold_time_sk as time_sk  
+                 from web_sales,date_dim
+                 where d_date_sk = ws_sold_date_sk
+                   and d_moy=12
+                   and d_year=2001
+                 union all
+                 select cs_ext_sales_price as ext_price,
+                        cs_sold_date_sk as sold_date_sk,
+                        cs_item_sk as sold_item_sk,
+                        cs_sold_time_sk as time_sk
+                 from catalog_sales,date_dim
+                 where d_date_sk = cs_sold_date_sk
+                   and d_moy=12
+                   and d_year=2001
+                 union all
+                 select ss_ext_sales_price as ext_price,
+                        ss_sold_date_sk as sold_date_sk,
+                        ss_item_sk as sold_item_sk,
+                        ss_sold_time_sk as time_sk
+                 from store_sales,date_dim
+                 where d_date_sk = ss_sold_date_sk
+                   and d_moy=12
+                   and d_year=2001
+                 ) as tmp,time_dim
+ where
+   sold_item_sk = i_item_sk
+   and i_manager_id=1
+   and time_sk = t_time_sk
+   and (t_meal_time = 'breakfast' or t_meal_time = 'dinner')
+ group by i_brand, i_brand_id,t_hour,t_minute
+ order by ext_price desc, i_brand_id
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select i_brand_id brand_id, i_brand brand,t_hour,t_minute, sum(ext_price) ext_price from item JOIN (select ws_ext_sales_price as ext_price, ws_sold_date_sk as sold_date_sk, ws_item_sk as sold_item_sk, ws_sold_time_sk as time_sk from web_sales,date_dim where date_dim.d_date_sk = web_sales.ws_sold_date_sk and d_moy=12 and d_year=2001 union all select cs_ext_sales_price as ext_price, cs_sold_date_sk as sold_date_sk, cs_item_sk as sold_item_sk, cs_sold_time_sk as time_sk from catalog_sales,date_dim where date_dim.d_date_sk = catalog_sales.cs_sold_date_sk and d_moy=12 and d_year=2001 union all select ss_ext_sales_price as ext_price, ss_sold_date_sk as sold_date_sk, ss_item_sk as sold_item_sk, ss_sold_time_sk as time_sk from store_sales,date_dim where date_dim.d_date_sk = store_sales.ss_sold_date_sk and d_moy=12 and d_year=2001 ) tmp ON tmp.sold_item_sk = item.i_item_sk JOIN time_dim ON tmp.time_sk = time_dim.t_time_sk where i_manager_id=1 and (t_meal_time = 'brea
 kfast' or t_meal_time = 'dinner') group by i_brand, i_brand_id,t_hour,t_minute order by ext_price desc, i_brand_id
+POSTHOOK: query: explain
+select i_brand_id brand_id, i_brand brand,t_hour,t_minute,
+ 	sum(ext_price) ext_price
+ from item, (select ws_ext_sales_price as ext_price, 
+                        ws_sold_date_sk as sold_date_sk,
+                        ws_item_sk as sold_item_sk,
+                        ws_sold_time_sk as time_sk  
+                 from web_sales,date_dim
+                 where d_date_sk = ws_sold_date_sk
+                   and d_moy=12
+                   and d_year=2001
+                 union all
+                 select cs_ext_sales_price as ext_price,
+                        cs_sold_date_sk as sold_date_sk,
+                        cs_item_sk as sold_item_sk,
+                        cs_sold_time_sk as time_sk
+                 from catalog_sales,date_dim
+                 where d_date_sk = cs_sold_date_sk
+                   and d_moy=12
+                   and d_year=2001
+                 union all
+                 select ss_ext_sales_price as ext_price,
+                        ss_sold_date_sk as sold_date_sk,
+                        ss_item_sk as sold_item_sk,
+                        ss_sold_time_sk as time_sk
+                 from store_sales,date_dim
+                 where d_date_sk = ss_sold_date_sk
+                   and d_moy=12
+                   and d_year=2001
+                 ) as tmp,time_dim
+ where
+   sold_item_sk = i_item_sk
+   and i_manager_id=1
+   and time_sk = t_time_sk
+   and (t_meal_time = 'breakfast' or t_meal_time = 'dinner')
+ group by i_brand, i_brand_id,t_hour,t_minute
+ order by ext_price desc, i_brand_id
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query73.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query73.q.out b/ql/src/test/results/clientpositive/perf/query73.q.out
index 9e1a39a..db8dfc0 100644
--- a/ql/src/test/results/clientpositive/perf/query73.q.out
+++ b/ql/src/test/results/clientpositive/perf/query73.q.out
@@ -1,6 +1,56 @@
-PREHOOK: query: explain select c_last_name ,c_first_name ,c_salutation ,c_preferred_cust_flag ,ss_ticket_number ,cnt from (select ss_ticket_number ,ss_customer_sk ,count(*) cnt from store_sales,date_dim,store,household_demographics where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and date_dim.d_dom between 1 and 2 and (household_demographics.hd_buy_potential = '1001-5000' or household_demographics.hd_buy_potential = '5001-10000') and household_demographics.hd_vehicle_count > 0 and case when household_demographics.hd_vehicle_count > 0 then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 and date_dim.d_year in (1998,1998+1,1998+2) and store.s_county in ('Kittitas County','Adams County','Richland County','Furnas County') group by ss_ticket_number,ss_customer_sk) dj,customer where dj.ss_customer_sk = customer.c_customer_sk an
 d cnt between 1 and 5 order by cnt desc
+PREHOOK: query: explain
+select c_last_name
+       ,c_first_name
+       ,c_salutation
+       ,c_preferred_cust_flag 
+       ,ss_ticket_number
+       ,cnt from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,count(*) cnt
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and date_dim.d_dom between 1 and 2 
+    and (household_demographics.hd_buy_potential = '>10000' or
+         household_demographics.hd_buy_potential = 'unknown')
+    and household_demographics.hd_vehicle_count > 0
+    and case when household_demographics.hd_vehicle_count > 0 then 
+             household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1
+    and date_dim.d_year in (2000,2000+1,2000+2)
+    and store.s_county in ('Mobile County','Maverick County','Huron County','Kittitas County')
+    group by ss_ticket_number,ss_customer_sk) dj,customer
+    where ss_customer_sk = c_customer_sk
+      and cnt between 1 and 5
+    order by cnt desc
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select c_last_name ,c_first_name ,c_salutation ,c_preferred_cust_flag ,ss_ticket_number ,cnt from (select ss_ticket_number ,ss_customer_sk ,count(*) cnt from store_sales,date_dim,store,household_demographics where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and date_dim.d_dom between 1 and 2 and (household_demographics.hd_buy_potential = '1001-5000' or household_demographics.hd_buy_potential = '5001-10000') and household_demographics.hd_vehicle_count > 0 and case when household_demographics.hd_vehicle_count > 0 then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 and date_dim.d_year in (1998,1998+1,1998+2) and store.s_county in ('Kittitas County','Adams County','Richland County','Furnas County') group by ss_ticket_number,ss_customer_sk) dj,customer where dj.ss_customer_sk = customer.c_customer_sk a
 nd cnt between 1 and 5 order by cnt desc
+POSTHOOK: query: explain
+select c_last_name
+       ,c_first_name
+       ,c_salutation
+       ,c_preferred_cust_flag 
+       ,ss_ticket_number
+       ,cnt from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,count(*) cnt
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and date_dim.d_dom between 1 and 2 
+    and (household_demographics.hd_buy_potential = '>10000' or
+         household_demographics.hd_buy_potential = 'unknown')
+    and household_demographics.hd_vehicle_count > 0
+    and case when household_demographics.hd_vehicle_count > 0 then 
+             household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1
+    and date_dim.d_year in (2000,2000+1,2000+2)
+    and store.s_county in ('Mobile County','Maverick County','Huron County','Kittitas County')
+    group by ss_ticket_number,ss_customer_sk) dj,customer
+    where ss_customer_sk = c_customer_sk
+      and cnt between 1 and 5
+    order by cnt desc
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -57,7 +107,7 @@ Stage-0
                                 Select Operator [SEL_14] (rows=852 width=1910)
                                   Output:["_col0"]
                                   Filter Operator [FIL_56] (rows=852 width=1910)
-                                    predicate:((s_county) IN ('Kittitas County', 'Adams County', 'Richland County', 'Furnas County') and s_store_sk is not null)
+                                    predicate:((s_county) IN ('Mobile County', 'Maverick County', 'Huron County', 'Kittitas County') and s_store_sk is not null)
                                     TableScan [TS_12] (rows=1704 width=1910)
                                       default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_county"]
                             <-Reducer 6 [SIMPLE_EDGE]
@@ -71,7 +121,7 @@ Stage-0
                                     Select Operator [SEL_11] (rows=1200 width=107)
                                       Output:["_col0"]
                                       Filter Operator [FIL_55] (rows=1200 width=107)
-                                        predicate:(((hd_buy_potential = '1001-5000') or (hd_buy_potential = '5001-10000')) and (hd_vehicle_count > 0) and CASE WHEN ((hd_vehicle_count > 0)) THEN (((UDFToDouble(hd_dep_count) / UDFToDouble(hd_vehicle_count)) > 1.0)) ELSE (null) END and hd_demo_sk is not null)
+                                        predicate:(((hd_buy_potential = '>10000') or (hd_buy_potential = 'unknown')) and (hd_vehicle_count > 0) and CASE WHEN ((hd_vehicle_count > 0)) THEN (((UDFToDouble(hd_dep_count) / UDFToDouble(hd_vehicle_count)) > 1.0)) ELSE (null) END and hd_demo_sk is not null)
                                         TableScan [TS_9] (rows=7200 width=107)
                                           default@household_demographics,household_demographics,Tbl:COMPLETE,Col:NONE,Output:["hd_demo_sk","hd_buy_potential","hd_dep_count","hd_vehicle_count"]
                                 <-Reducer 5 [SIMPLE_EDGE]
@@ -94,7 +144,7 @@ Stage-0
                                         Select Operator [SEL_8] (rows=4058 width=1119)
                                           Output:["_col0"]
                                           Filter Operator [FIL_54] (rows=4058 width=1119)
-                                            predicate:((d_year) IN (1998, 1999, 2000) and d_dom BETWEEN 1 AND 2 and d_date_sk is not null)
+                                            predicate:((d_year) IN (2000, 2001, 2002) and d_dom BETWEEN 1 AND 2 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_dom"]
 


[14/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query78.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query78.q b/ql/src/test/queries/clientpositive/perf/query78.q
new file mode 100644
index 0000000..ca9e6d6
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query78.q
@@ -0,0 +1,60 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query78.tpl and seed 1819994127
+explain
+with ws as
+  (select d_year AS ws_sold_year, ws_item_sk,
+    ws_bill_customer_sk ws_customer_sk,
+    sum(ws_quantity) ws_qty,
+    sum(ws_wholesale_cost) ws_wc,
+    sum(ws_sales_price) ws_sp
+   from web_sales
+   left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk
+   join date_dim on ws_sold_date_sk = d_date_sk
+   where wr_order_number is null
+   group by d_year, ws_item_sk, ws_bill_customer_sk
+   ),
+cs as
+  (select d_year AS cs_sold_year, cs_item_sk,
+    cs_bill_customer_sk cs_customer_sk,
+    sum(cs_quantity) cs_qty,
+    sum(cs_wholesale_cost) cs_wc,
+    sum(cs_sales_price) cs_sp
+   from catalog_sales
+   left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk
+   join date_dim on cs_sold_date_sk = d_date_sk
+   where cr_order_number is null
+   group by d_year, cs_item_sk, cs_bill_customer_sk
+   ),
+ss as
+  (select d_year AS ss_sold_year, ss_item_sk,
+    ss_customer_sk,
+    sum(ss_quantity) ss_qty,
+    sum(ss_wholesale_cost) ss_wc,
+    sum(ss_sales_price) ss_sp
+   from store_sales
+   left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+   join date_dim on ss_sold_date_sk = d_date_sk
+   where sr_ticket_number is null
+   group by d_year, ss_item_sk, ss_customer_sk
+   )
+ select 
+ss_sold_year, ss_item_sk, ss_customer_sk,
+round(ss_qty/(coalesce(ws_qty+cs_qty,1)),2) ratio,
+ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price,
+coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty,
+coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost,
+coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price
+from ss
+left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk)
+left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=cs_item_sk and cs_customer_sk=ss_customer_sk)
+where coalesce(ws_qty,0)>0 and coalesce(cs_qty, 0)>0 and ss_sold_year=2000
+order by 
+  ss_sold_year, ss_item_sk, ss_customer_sk,
+  ss_qty desc, ss_wc desc, ss_sp desc,
+  other_chan_qty,
+  other_chan_wholesale_cost,
+  other_chan_sales_price,
+  round(ss_qty/(coalesce(ws_qty+cs_qty,1)),2)
+limit 100;
+
+-- end query 1 in stream 0 using template query78.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query79.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query79.q b/ql/src/test/queries/clientpositive/perf/query79.q
index d104602..dfa7017 100644
--- a/ql/src/test/queries/clientpositive/perf/query79.q
+++ b/ql/src/test/queries/clientpositive/perf/query79.q
@@ -1 +1,25 @@
-explain select c_last_name,c_first_name,substr(s_city,1,30) sub,ss_ticket_number,amt,profit from (select ss_ticket_number ,ss_customer_sk ,store.s_city ,sum(ss_coupon_amt) amt ,sum(ss_net_profit) profit from store_sales,date_dim,store,household_demographics where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and (household_demographics.hd_dep_count = 8 or household_demographics.hd_vehicle_count > 0) and date_dim.d_dow = 1 and date_dim.d_year in (1998,1998+1,1998+2) and store.s_number_employees between 200 and 295 group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer where ms.ss_customer_sk = customer.c_customer_sk order by c_last_name,c_first_name,sub, profit limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query79.tpl and seed 2031708268
+explain
+select 
+  c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit
+  from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,store.s_city
+          ,sum(ss_coupon_amt) amt
+          ,sum(ss_net_profit) profit
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and (household_demographics.hd_dep_count = 8 or household_demographics.hd_vehicle_count > 0)
+    and date_dim.d_dow = 1
+    and date_dim.d_year in (1998,1998+1,1998+2) 
+    and store.s_number_employees between 200 and 295
+    group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer
+    where ss_customer_sk = c_customer_sk
+ order by c_last_name,c_first_name,substr(s_city,1,30), profit
+limit 100;
+
+-- end query 1 in stream 0 using template query79.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query8.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query8.q b/ql/src/test/queries/clientpositive/perf/query8.q
index bb77f55..cfce366 100644
--- a/ql/src/test/queries/clientpositive/perf/query8.q
+++ b/ql/src/test/queries/clientpositive/perf/query8.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query8.tpl and seed 1766988859
 explain
 select  s_store_name
       ,sum(ss_net_profit)
@@ -105,3 +107,4 @@ select  s_store_name
  order by s_store_name
  limit 100;
 
+-- end query 1 in stream 0 using template query8.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query80.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query80.q b/ql/src/test/queries/clientpositive/perf/query80.q
index 53f9a3c..651c5d7 100644
--- a/ql/src/test/queries/clientpositive/perf/query80.q
+++ b/ql/src/test/queries/clientpositive/perf/query80.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query80.tpl and seed 1819994127
 explain
 with ssr as
  (select  s_store_id as store_id,
@@ -93,3 +95,4 @@ group by web_site_id)
          ,id
  limit 100;
 
+-- end query 1 in stream 0 using template query80.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query81.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query81.q b/ql/src/test/queries/clientpositive/perf/query81.q
index 77de348..fd072c3 100644
--- a/ql/src/test/queries/clientpositive/perf/query81.q
+++ b/ql/src/test/queries/clientpositive/perf/query81.q
@@ -1,6 +1,7 @@
 set hive.mapred.mode=nonstrict;
-
-explain with customer_total_return as
+-- start query 1 in stream 0 using template query81.tpl and seed 1819994127
+explain
+with customer_total_return as
  (select cr_returning_customer_sk as ctr_customer_sk
         ,ca_state as ctr_state, 
  	sum(cr_return_amt_inc_tax) as ctr_total_return
@@ -28,3 +29,5 @@ explain with customer_total_return as
                    ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset
                   ,ca_location_type,ctr_total_return
  limit 100;
+
+-- end query 1 in stream 0 using template query81.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query82.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query82.q b/ql/src/test/queries/clientpositive/perf/query82.q
index cf882b8..9aec0cb 100644
--- a/ql/src/test/queries/clientpositive/perf/query82.q
+++ b/ql/src/test/queries/clientpositive/perf/query82.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query82.tpl and seed 55585014
 explain
 select  i_item_id
        ,i_item_desc
@@ -14,3 +16,4 @@ select  i_item_id
  order by i_item_id
  limit 100;
 
+-- end query 1 in stream 0 using template query82.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query83.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query83.q b/ql/src/test/queries/clientpositive/perf/query83.q
index 0186e03..fd9184c 100644
--- a/ql/src/test/queries/clientpositive/perf/query83.q
+++ b/ql/src/test/queries/clientpositive/perf/query83.q
@@ -1,5 +1,7 @@
+set hive.mapred.mode=nonstrict;
 -- start query 1 in stream 0 using template query83.tpl and seed 1930872976
-explain with sr_items as
+explain
+with sr_items as
  (select i_item_id item_id,
         sum(sr_return_quantity) sr_item_qty
  from store_returns,
@@ -63,3 +65,5 @@ explain with sr_items as
  order by sr_items.item_id
          ,sr_item_qty
  limit 100;
+
+-- end query 1 in stream 0 using template query83.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query84.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query84.q b/ql/src/test/queries/clientpositive/perf/query84.q
index bcc2f4e..4ab5945 100644
--- a/ql/src/test/queries/clientpositive/perf/query84.q
+++ b/ql/src/test/queries/clientpositive/perf/query84.q
@@ -1 +1,23 @@
-explain select c_customer_id as customer_id ,concat(c_last_name, ', ', c_first_name) as customername from customer ,customer_address ,customer_demographics ,household_demographics ,income_band ,store_returns where ca_city = 'Hopewell' and customer.c_current_addr_sk = customer_address.ca_address_sk and ib_lower_bound >= 32287 and ib_upper_bound <= 32287 + 50000 and income_band.ib_income_band_sk = household_demographics.hd_income_band_sk and customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk and household_demographics.hd_demo_sk = customer.c_current_hdemo_sk and store_returns.sr_cdemo_sk = customer_demographics.cd_demo_sk order by customer_id limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query84.tpl and seed 1819994127
+explain
+select  c_customer_id as customer_id
+       ,c_last_name || ', ' || c_first_name as customername
+ from customer
+     ,customer_address
+     ,customer_demographics
+     ,household_demographics
+     ,income_band
+     ,store_returns
+ where ca_city	        =  'Hopewell'
+   and c_current_addr_sk = ca_address_sk
+   and ib_lower_bound   >=  32287
+   and ib_upper_bound   <=  32287 + 50000
+   and ib_income_band_sk = hd_income_band_sk
+   and cd_demo_sk = c_current_cdemo_sk
+   and hd_demo_sk = c_current_hdemo_sk
+   and sr_cdemo_sk = cd_demo_sk
+ order by c_customer_id
+ limit 100;
+
+-- end query 1 in stream 0 using template query84.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query85.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query85.q b/ql/src/test/queries/clientpositive/perf/query85.q
index b1e2b64..2e67e72 100644
--- a/ql/src/test/queries/clientpositive/perf/query85.q
+++ b/ql/src/test/queries/clientpositive/perf/query85.q
@@ -1 +1,86 @@
-explain select substr(r_reason_desc,1,20) as r ,avg(ws_quantity) wq ,avg(wr_refunded_cash) ref ,avg(wr_fee) fee from web_sales, web_returns, web_page, customer_demographics cd1, customer_demographics cd2, customer_address, date_dim, reason where web_sales.ws_web_page_sk = web_page.wp_web_page_sk and web_sales.ws_item_sk = web_returns.wr_item_sk and web_sales.ws_order_number = web_returns.wr_order_number and web_sales.ws_sold_date_sk = date_dim.d_date_sk and d_year = 1998 and cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk and cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk and customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk and reason.r_reason_sk = web_returns.wr_reason_sk and ( ( cd1.cd_marital_status = 'M' and cd1.cd_marital_status = cd2.cd_marital_status and cd1.cd_education_status = '4 yr Degree' and cd1.cd_education_status = cd2.cd_education_status and ws_sales_price between 100.00 and 150.00 ) or ( cd1.cd_marital_status = 'D' and cd1.cd_marital_status = c
 d2.cd_marital_status and cd1.cd_education_status = 'Primary' and cd1.cd_education_status = cd2.cd_education_status and ws_sales_price between 50.00 and 100.00 ) or ( cd1.cd_marital_status = 'U' and cd1.cd_marital_status = cd2.cd_marital_status and cd1.cd_education_status = 'Advanced Degree' and cd1.cd_education_status = cd2.cd_education_status and ws_sales_price between 150.00 and 200.00 ) ) and ( ( ca_country = 'United States' and ca_state in ('KY', 'GA', 'NM') and ws_net_profit between 100 and 200 ) or ( ca_country = 'United States' and ca_state in ('MT', 'OR', 'IN') and ws_net_profit between 150 and 300 ) or ( ca_country = 'United States' and ca_state in ('WI', 'MO', 'WV') and ws_net_profit between 50 and 250 ) ) group by r_reason_desc order by r, wq, ref, fee limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query85.tpl and seed 622697896
+explain
+select  substr(r_reason_desc,1,20)
+       ,avg(ws_quantity)
+       ,avg(wr_refunded_cash)
+       ,avg(wr_fee)
+ from web_sales, web_returns, web_page, customer_demographics cd1,
+      customer_demographics cd2, customer_address, date_dim, reason 
+ where ws_web_page_sk = wp_web_page_sk
+   and ws_item_sk = wr_item_sk
+   and ws_order_number = wr_order_number
+   and ws_sold_date_sk = d_date_sk and d_year = 1998
+   and cd1.cd_demo_sk = wr_refunded_cdemo_sk 
+   and cd2.cd_demo_sk = wr_returning_cdemo_sk
+   and ca_address_sk = wr_refunded_addr_sk
+   and r_reason_sk = wr_reason_sk
+   and
+   (
+    (
+     cd1.cd_marital_status = 'M'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = '4 yr Degree'
+     and 
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 100.00 and 150.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'D'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Primary' 
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 50.00 and 100.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'U'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Advanced Degree'
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 150.00 and 200.00
+    )
+   )
+   and
+   (
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('KY', 'GA', 'NM')
+     and ws_net_profit between 100 and 200  
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('MT', 'OR', 'IN')
+     and ws_net_profit between 150 and 300  
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('WI', 'MO', 'WV')
+     and ws_net_profit between 50 and 250  
+    )
+   )
+group by r_reason_desc
+order by substr(r_reason_desc,1,20)
+        ,avg(ws_quantity)
+        ,avg(wr_refunded_cash)
+        ,avg(wr_fee)
+limit 100;
+
+-- end query 1 in stream 0 using template query85.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query86.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query86.q b/ql/src/test/queries/clientpositive/perf/query86.q
index 07a9ec5..6670868 100644
--- a/ql/src/test/queries/clientpositive/perf/query86.q
+++ b/ql/src/test/queries/clientpositive/perf/query86.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query86.tpl and seed 1819994127
 explain
 select   
     sum(ws_net_paid) as total_sum
@@ -5,9 +7,9 @@ select
    ,i_class
    ,grouping(i_category)+grouping(i_class) as lochierarchy
    ,rank() over (
-     partition by grouping(i_category)+grouping(i_class),
-     case when grouping(i_class) = 0 then i_category end 
-     order by sum(ws_net_paid) desc) as rank_within_parent
+ 	partition by grouping(i_category)+grouping(i_class),
+ 	case when grouping(i_class) = 0 then i_category end 
+ 	order by sum(ws_net_paid) desc) as rank_within_parent
  from
     web_sales
    ,date_dim       d1
@@ -23,3 +25,4 @@ select
    rank_within_parent
  limit 100;
 
+-- end query 1 in stream 0 using template query86.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query87.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query87.q b/ql/src/test/queries/clientpositive/perf/query87.q
index fe12ee3..e4562c2 100644
--- a/ql/src/test/queries/clientpositive/perf/query87.q
+++ b/ql/src/test/queries/clientpositive/perf/query87.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query87.tpl and seed 1819994127
 explain
 select count(*) 
 from ((select distinct c_last_name, c_first_name, d_date
@@ -20,3 +22,4 @@ from ((select distinct c_last_name, c_first_name, d_date
 ) cool_cust
 ;
 
+-- end query 1 in stream 0 using template query87.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query88.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query88.q b/ql/src/test/queries/clientpositive/perf/query88.q
index bb6ef6d..265cc7c 100644
--- a/ql/src/test/queries/clientpositive/perf/query88.q
+++ b/ql/src/test/queries/clientpositive/perf/query88.q
@@ -1,13 +1,13 @@
-set hive.strict.checks.cartesian.product=false;
-
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query88.tpl and seed 318176889
 explain
 select  *
 from
  (select count(*) h8_30_to_9
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk   
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk 
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk   
+     and ss_hdemo_sk = household_demographics.hd_demo_sk 
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 8
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -16,9 +16,9 @@ from
      and store.s_store_name = 'ese') s1,
  (select count(*) h9_to_9_30 
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk 
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk 
      and time_dim.t_hour = 9 
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -27,9 +27,9 @@ from
      and store.s_store_name = 'ese') s2,
  (select count(*) h9_30_to_10 
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 9
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -38,9 +38,9 @@ from
      and store.s_store_name = 'ese') s3,
  (select count(*) h10_to_10_30
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 10 
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -49,9 +49,9 @@ from
      and store.s_store_name = 'ese') s4,
  (select count(*) h10_30_to_11
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 10 
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -60,9 +60,9 @@ from
      and store.s_store_name = 'ese') s5,
  (select count(*) h11_to_11_30
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk 
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk 
      and time_dim.t_hour = 11
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -71,9 +71,9 @@ from
      and store.s_store_name = 'ese') s6,
  (select count(*) h11_30_to_12
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 11
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -82,9 +82,9 @@ from
      and store.s_store_name = 'ese') s7,
  (select count(*) h12_to_12_30
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 12
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -92,3 +92,5 @@ from
           (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
      and store.s_store_name = 'ese') s8
 ;
+
+-- end query 1 in stream 0 using template query88.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query89.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query89.q b/ql/src/test/queries/clientpositive/perf/query89.q
index a628c0e..3159229 100644
--- a/ql/src/test/queries/clientpositive/perf/query89.q
+++ b/ql/src/test/queries/clientpositive/perf/query89.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query89.tpl and seed 1719819282
 explain
 select  *
 from(
@@ -9,9 +11,9 @@ select i_category, i_class, i_brand,
          (partition by i_category, i_brand, s_store_name, s_company_name)
          avg_monthly_sales
 from item, store_sales, date_dim, store
-where store_sales.ss_item_sk = item.i_item_sk and
-      store_sales.ss_sold_date_sk = date_dim.d_date_sk and
-      store_sales.ss_store_sk = store.s_store_sk and
+where ss_item_sk = i_item_sk and
+      ss_sold_date_sk = d_date_sk and
+      ss_store_sk = s_store_sk and
       d_year in (2000) and
         ((i_category in ('Home','Books','Electronics') and
           i_class in ('wallpaper','parenting','musical')
@@ -25,3 +27,4 @@ where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales
 order by sum_sales - avg_monthly_sales, s_store_name
 limit 100;
 
+-- end query 1 in stream 0 using template query89.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query9.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query9.q b/ql/src/test/queries/clientpositive/perf/query9.q
index 77bffca..421f5e1 100644
--- a/ql/src/test/queries/clientpositive/perf/query9.q
+++ b/ql/src/test/queries/clientpositive/perf/query9.q
@@ -1,11 +1,12 @@
 set hive.mapred.mode=nonstrict;
-
-explain select case when (select count(*)
-                  from store_sales
+-- start query 1 in stream 0 using template query9.tpl and seed 1490436826
+explain
+select case when (select count(*) 
+                  from store_sales 
                   where ss_quantity between 1 and 20) > 409437
-            then (select avg(ss_ext_list_price)
-                  from store_sales
-                  where ss_quantity between 1 and 20)
+            then (select avg(ss_ext_list_price) 
+                  from store_sales 
+                  where ss_quantity between 1 and 20) 
             else (select avg(ss_net_paid_inc_tax)
                   from store_sales
                   where ss_quantity between 1 and 20) end bucket1 ,
@@ -14,7 +15,7 @@ explain select case when (select count(*)
                   where ss_quantity between 21 and 40) > 4595804
             then (select avg(ss_ext_list_price)
                   from store_sales
-                  where ss_quantity between 21 and 40)
+                  where ss_quantity between 21 and 40) 
             else (select avg(ss_net_paid_inc_tax)
                   from store_sales
                   where ss_quantity between 21 and 40) end bucket2,
@@ -47,4 +48,6 @@ explain select case when (select count(*)
                   where ss_quantity between 81 and 100) end bucket5
 from reason
 where r_reason_sk = 1
-;
\ No newline at end of file
+;
+
+-- end query 1 in stream 0 using template query9.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query90.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query90.q b/ql/src/test/queries/clientpositive/perf/query90.q
index 515da06..d17cbc4 100644
--- a/ql/src/test/queries/clientpositive/perf/query90.q
+++ b/ql/src/test/queries/clientpositive/perf/query90.q
@@ -1,2 +1,24 @@
 set hive.mapred.mode=nonstrict;
-explain select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio from ( select count(*) amc from web_sales, household_demographics , time_dim, web_page where ws_sold_time_sk = time_dim.t_time_sk and ws_ship_hdemo_sk = household_demographics.hd_demo_sk and ws_web_page_sk = web_page.wp_web_page_sk and time_dim.t_hour between 6 and 6+1 and household_demographics.hd_dep_count = 8 and web_page.wp_char_count between 5000 and 5200) at, ( select count(*) pmc from web_sales, household_demographics , time_dim, web_page where ws_sold_time_sk = time_dim.t_time_sk and ws_ship_hdemo_sk = household_demographics.hd_demo_sk and ws_web_page_sk = web_page.wp_web_page_sk and time_dim.t_hour between 14 and 14+1 and household_demographics.hd_dep_count = 8 and web_page.wp_char_count between 5000 and 5200) pt order by am_pm_ratio limit 100;
+-- start query 1 in stream 0 using template query90.tpl and seed 2031708268
+explain
+select  cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio
+ from ( select count(*) amc
+       from web_sales, household_demographics , time_dim, web_page
+       where ws_sold_time_sk = time_dim.t_time_sk
+         and ws_ship_hdemo_sk = household_demographics.hd_demo_sk
+         and ws_web_page_sk = web_page.wp_web_page_sk
+         and time_dim.t_hour between 6 and 6+1
+         and household_demographics.hd_dep_count = 8
+         and web_page.wp_char_count between 5000 and 5200) at,
+      ( select count(*) pmc
+       from web_sales, household_demographics , time_dim, web_page
+       where ws_sold_time_sk = time_dim.t_time_sk
+         and ws_ship_hdemo_sk = household_demographics.hd_demo_sk
+         and ws_web_page_sk = web_page.wp_web_page_sk
+         and time_dim.t_hour between 14 and 14+1
+         and household_demographics.hd_dep_count = 8
+         and web_page.wp_char_count between 5000 and 5200) pt
+ order by am_pm_ratio
+ limit 100;
+
+-- end query 1 in stream 0 using template query90.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query91.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query91.q b/ql/src/test/queries/clientpositive/perf/query91.q
index 9458c0c..79ca713 100644
--- a/ql/src/test/queries/clientpositive/perf/query91.q
+++ b/ql/src/test/queries/clientpositive/perf/query91.q
@@ -1,2 +1,33 @@
 set hive.mapred.mode=nonstrict;
-explain select cc_call_center_id Call_Center, cc_name Call_Center_Name, cc_manager Manager, sum(cr_net_loss) Returns_Loss from call_center, catalog_returns, date_dim, customer, customer_address, customer_demographics, household_demographics where catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk and catalog_returns.cr_returned_date_sk = date_dim.d_date_sk and catalog_returns.cr_returning_customer_sk= customer.c_customer_sk and customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk and household_demographics.hd_demo_sk = customer.c_current_hdemo_sk and customer_address.ca_address_sk = customer.c_current_addr_sk and d_year = 1999 and d_moy = 11 and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) and hd_buy_potential like '0-500%' and ca_gmt_offset = -7 group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status order by Returns_Loss desc;
+-- start query 1 in stream 0 using template query91.tpl and seed 1930872976
+explain
+select  
+        cc_call_center_id Call_Center,
+        cc_name Call_Center_Name,
+        cc_manager Manager,
+        sum(cr_net_loss) Returns_Loss
+from
+        call_center,
+        catalog_returns,
+        date_dim,
+        customer,
+        customer_address,
+        customer_demographics,
+        household_demographics
+where
+        cr_call_center_sk       = cc_call_center_sk
+and     cr_returned_date_sk     = d_date_sk
+and     cr_returning_customer_sk= c_customer_sk
+and     cd_demo_sk              = c_current_cdemo_sk
+and     hd_demo_sk              = c_current_hdemo_sk
+and     ca_address_sk           = c_current_addr_sk
+and     d_year                  = 1999 
+and     d_moy                   = 11
+and     ( (cd_marital_status       = 'M' and cd_education_status     = 'Unknown')
+        or(cd_marital_status       = 'W' and cd_education_status     = 'Advanced Degree'))
+and     hd_buy_potential like '0-500%'
+and     ca_gmt_offset           = -7
+group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status
+order by sum(cr_net_loss) desc;
+
+-- end query 1 in stream 0 using template query91.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query92.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query92.q b/ql/src/test/queries/clientpositive/perf/query92.q
index 625e99f..f26fa5e 100644
--- a/ql/src/test/queries/clientpositive/perf/query92.q
+++ b/ql/src/test/queries/clientpositive/perf/query92.q
@@ -1 +1,32 @@
-explain SELECT sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) as store_only, sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) as catalog_only, sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) as store_and_catalog FROM (SELECT ss.ss_customer_sk as customer_sk, ss.ss_item_sk as item_sk FROM store_sales ss JOIN date_dim d1 ON (ss.ss_sold_date_sk = d1.d_date_sk) WHERE d1.d_month_seq >= 1206 and d1.d_month_seq <= 1217 GROUP BY ss.ss_customer_sk, ss.ss_item_sk) ssci FULL OUTER JOIN (SELECT cs.cs_bill_customer_sk as customer_sk, cs.cs_item_sk as item_sk FROM catalog_sales cs JOIN date_dim d2 ON (cs.cs_sold_date_sk = d2.d_date_sk) WHERE d2.d_month_seq >= 1206 and d2.d_month_seq <= 1217 GROUP BY cs.cs_bill_customer_sk, cs.cs_item_sk) csci ON (ssci.customer_sk=csci.customer_sk and ssci.item_sk = csci.item_sk);
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query92.tpl and seed 2031708268
+explain
+select  
+   sum(ws_ext_discount_amt)  as `Excess Discount Amount` 
+from 
+    web_sales 
+   ,item 
+   ,date_dim
+where
+i_manufact_id = 269
+and i_item_sk = ws_item_sk 
+and d_date between '1998-03-18' and 
+        (cast('1998-03-18' as date) + 90 days)
+and d_date_sk = ws_sold_date_sk 
+and ws_ext_discount_amt  
+     > ( 
+         SELECT 
+            1.3 * avg(ws_ext_discount_amt) 
+         FROM 
+            web_sales 
+           ,date_dim
+         WHERE 
+              ws_item_sk = i_item_sk 
+          and d_date between '1998-03-18' and
+                             (cast('1998-03-18' as date) + 90 days)
+          and d_date_sk = ws_sold_date_sk 
+      ) 
+order by sum(ws_ext_discount_amt)
+limit 100;
+
+-- end query 1 in stream 0 using template query92.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query93.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query93.q b/ql/src/test/queries/clientpositive/perf/query93.q
index b60b041..7f4a093 100644
--- a/ql/src/test/queries/clientpositive/perf/query93.q
+++ b/ql/src/test/queries/clientpositive/perf/query93.q
@@ -1 +1,20 @@
-explain select ss_customer_sk ,sum(act_sales) sumsales from (select ss_item_sk ,ss_ticket_number ,ss_customer_sk ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price else (ss_quantity*ss_sales_price) end act_sales from store_sales left outer join store_returns on (store_returns.sr_item_sk = store_sales.ss_item_sk and store_returns.sr_ticket_number = store_sales.ss_ticket_number) ,reason where store_returns.sr_reason_sk = reason.r_reason_sk and r_reason_desc = 'Did not like the warranty') t group by ss_customer_sk order by sumsales, ss_customer_sk limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query93.tpl and seed 1200409435
+explain
+select  ss_customer_sk
+            ,sum(act_sales) sumsales
+      from (select ss_item_sk
+                  ,ss_ticket_number
+                  ,ss_customer_sk
+                  ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price
+                                                            else (ss_quantity*ss_sales_price) end act_sales
+            from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk
+                                                               and sr_ticket_number = ss_ticket_number)
+                ,reason
+            where sr_reason_sk = r_reason_sk
+              and r_reason_desc = 'Did not like the warranty') t
+      group by ss_customer_sk
+      order by sumsales, ss_customer_sk
+limit 100;
+
+-- end query 1 in stream 0 using template query93.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query94.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query94.q b/ql/src/test/queries/clientpositive/perf/query94.q
index f9f4bc1..18253fa 100644
--- a/ql/src/test/queries/clientpositive/perf/query94.q
+++ b/ql/src/test/queries/clientpositive/perf/query94.q
@@ -1 +1,31 @@
-explain SELECT count(distinct ws_order_number) as order_count, sum(ws_ext_ship_cost) as total_shipping_cost, sum(ws_net_profit) as total_net_profit FROM web_sales ws1 JOIN customer_address ca ON (ws1.ws_ship_addr_sk = ca.ca_address_sk) JOIN web_site s ON (ws1.ws_web_site_sk = s.web_site_sk) JOIN date_dim d ON (ws1.ws_ship_date_sk = d.d_date_sk) LEFT SEMI JOIN (SELECT ws2.ws_order_number as ws_order_number FROM web_sales ws2 JOIN web_sales ws3 ON (ws2.ws_order_number = ws3.ws_order_number) WHERE ws2.ws_warehouse_sk <> ws3.ws_warehouse_sk ) ws_wh1 ON (ws1.ws_order_number = ws_wh1.ws_order_number) LEFT OUTER JOIN web_returns wr1 ON (ws1.ws_order_number = wr1.wr_order_number) WHERE d.d_date between '1999-05-01' and '1999-07-01' and ca.ca_state = 'TX' and s.web_company_name = 'pri' and wr1.wr_order_number is null limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query94.tpl and seed 2031708268
+explain
+select  
+   count(distinct ws_order_number) as `order count`
+  ,sum(ws_ext_ship_cost) as `total shipping cost`
+  ,sum(ws_net_profit) as `total net profit`
+from
+   web_sales ws1
+  ,date_dim
+  ,customer_address
+  ,web_site
+where
+    d_date between '1999-5-01' and 
+           (cast('1999-5-01' as date) + 60 days)
+and ws1.ws_ship_date_sk = d_date_sk
+and ws1.ws_ship_addr_sk = ca_address_sk
+and ca_state = 'TX'
+and ws1.ws_web_site_sk = web_site_sk
+and web_company_name = 'pri'
+and exists (select *
+            from web_sales ws2
+            where ws1.ws_order_number = ws2.ws_order_number
+              and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk)
+and not exists(select *
+               from web_returns wr1
+               where ws1.ws_order_number = wr1.wr_order_number)
+order by count(distinct ws_order_number)
+limit 100;
+
+-- end query 1 in stream 0 using template query94.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query95.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query95.q b/ql/src/test/queries/clientpositive/perf/query95.q
index fbd2d47..e9024a8 100644
--- a/ql/src/test/queries/clientpositive/perf/query95.q
+++ b/ql/src/test/queries/clientpositive/perf/query95.q
@@ -1 +1,34 @@
-explain SELECT count(distinct ws1.ws_order_number) as order_count, sum(ws1.ws_ext_ship_cost) as total_shipping_cost, sum(ws1.ws_net_profit) as total_net_profit FROM web_sales ws1 JOIN customer_address ca ON (ws1.ws_ship_addr_sk = ca.ca_address_sk) JOIN web_site s ON (ws1.ws_web_site_sk = s.web_site_sk) JOIN date_dim d ON (ws1.ws_ship_date_sk = d.d_date_sk) LEFT SEMI JOIN (SELECT ws2.ws_order_number as ws_order_number FROM web_sales ws2 JOIN web_sales ws3 ON (ws2.ws_order_number = ws3.ws_order_number) WHERE ws2.ws_warehouse_sk <> ws3.ws_warehouse_sk ) ws_wh1 ON (ws1.ws_order_number = ws_wh1.ws_order_number) LEFT SEMI JOIN (SELECT wr_order_number FROM web_returns wr JOIN (SELECT ws4.ws_order_number as ws_order_number FROM web_sales ws4 JOIN web_sales ws5 ON (ws4.ws_order_number = ws5.ws_order_number) WHERE ws4.ws_warehouse_sk <> ws5.ws_warehouse_sk ) ws_wh2 ON (wr.wr_order_number = ws_wh2.ws_order_number)) tmp1 ON (ws1.ws_order_number = tmp1.wr_order_number) WHERE d.d_date between '20
 02-05-01' and '2002-06-30' and ca.ca_state = 'GA' and s.web_company_name = 'pri';
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query95.tpl and seed 2031708268
+explain
+with ws_wh as
+(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2
+ from web_sales ws1,web_sales ws2
+ where ws1.ws_order_number = ws2.ws_order_number
+   and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk)
+ select  
+   count(distinct ws_order_number) as `order count`
+  ,sum(ws_ext_ship_cost) as `total shipping cost`
+  ,sum(ws_net_profit) as `total net profit`
+from
+   web_sales ws1
+  ,date_dim
+  ,customer_address
+  ,web_site
+where
+    d_date between '1999-5-01' and 
+           (cast('1999-5-01' as date) + 60 days)
+and ws1.ws_ship_date_sk = d_date_sk
+and ws1.ws_ship_addr_sk = ca_address_sk
+and ca_state = 'TX'
+and ws1.ws_web_site_sk = web_site_sk
+and web_company_name = 'pri'
+and ws1.ws_order_number in (select ws_order_number
+                            from ws_wh)
+and ws1.ws_order_number in (select wr_order_number
+                            from web_returns,ws_wh
+                            where wr_order_number = ws_wh.ws_order_number)
+order by count(distinct ws_order_number)
+limit 100;
+
+-- end query 1 in stream 0 using template query95.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query96.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query96.q b/ql/src/test/queries/clientpositive/perf/query96.q
index 29265ed..a306d6c 100644
--- a/ql/src/test/queries/clientpositive/perf/query96.q
+++ b/ql/src/test/queries/clientpositive/perf/query96.q
@@ -1 +1,18 @@
-explain select count(*) as c from store_sales ,household_demographics ,time_dim, store where store_sales.ss_sold_time_sk = time_dim.t_time_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and store_sales.ss_store_sk = store.s_store_sk and time_dim.t_hour = 8 and time_dim.t_minute >= 30 and household_demographics.hd_dep_count = 5 and store.s_store_name = 'ese' order by c limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query96.tpl and seed 1819994127
+explain
+select  count(*) 
+from store_sales
+    ,household_demographics 
+    ,time_dim, store
+where ss_sold_time_sk = time_dim.t_time_sk   
+    and ss_hdemo_sk = household_demographics.hd_demo_sk 
+    and ss_store_sk = s_store_sk
+    and time_dim.t_hour = 8
+    and time_dim.t_minute >= 30
+    and household_demographics.hd_dep_count = 5
+    and store.s_store_name = 'ese'
+order by count(*)
+limit 100;
+
+-- end query 1 in stream 0 using template query96.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query97.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query97.q b/ql/src/test/queries/clientpositive/perf/query97.q
index 4995309..7203e52 100644
--- a/ql/src/test/queries/clientpositive/perf/query97.q
+++ b/ql/src/test/queries/clientpositive/perf/query97.q
@@ -1 +1,27 @@
-explain select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog from ( select ss_customer_sk customer_sk ,ss_item_sk item_sk from store_sales JOIN date_dim ON store_sales.ss_sold_date_sk = date_dim.d_date_sk where d_month_seq between 1193 and 1193 + 11 group by ss_customer_sk ,ss_item_sk) ssci full outer join ( select cs_bill_customer_sk customer_sk ,cs_item_sk item_sk from catalog_sales JOIN date_dim ON catalog_sales.cs_sold_date_sk = date_dim.d_date_sk where d_month_seq between 1193 and 1193 + 11 group by cs_bill_customer_sk ,cs_item_sk) csci on (ssci.customer_sk=csci.customer_sk and ssci.item_sk = csci.item_sk) limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query97.tpl and seed 1819994127
+explain
+with ssci as (
+select ss_customer_sk customer_sk
+      ,ss_item_sk item_sk
+from store_sales,date_dim
+where ss_sold_date_sk = d_date_sk
+  and d_month_seq between 1212 and 1212 + 11
+group by ss_customer_sk
+        ,ss_item_sk),
+csci as(
+ select cs_bill_customer_sk customer_sk
+      ,cs_item_sk item_sk
+from catalog_sales,date_dim
+where cs_sold_date_sk = d_date_sk
+  and d_month_seq between 1212 and 1212 + 11
+group by cs_bill_customer_sk
+        ,cs_item_sk)
+ select  sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only
+      ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only
+      ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog
+from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk
+                               and ssci.item_sk = csci.item_sk)
+limit 100;
+
+-- end query 1 in stream 0 using template query97.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query98.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query98.q b/ql/src/test/queries/clientpositive/perf/query98.q
index ec4b553..6168f2a 100644
--- a/ql/src/test/queries/clientpositive/perf/query98.q
+++ b/ql/src/test/queries/clientpositive/perf/query98.q
@@ -1,7 +1,34 @@
 set hive.mapred.mode=nonstrict;
-explain 
-select i_item_desc ,i_category ,i_class ,i_current_price ,i_item_id ,sum(ss_ext_sales_price) as itemrevenue ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over (partition by i_class) as revenueratio from store_sales ,item ,date_dim 
-where store_sales.ss_item_sk = item.i_item_sk and i_category in ('Jewelry', 'Sports', 'Books') and store_sales.ss_sold_date_sk = date_dim.d_date_sk and 
-d_date between cast('2001-01-12' as date)
-                                and (cast('2001-01-12' as date) + 30 days)
-group by i_item_id ,i_item_desc ,i_category ,i_class ,i_current_price order by i_category ,i_class ,i_item_id ,i_item_desc ,revenueratio;
+-- start query 1 in stream 0 using template query98.tpl and seed 345591136
+explain
+select i_item_desc 
+      ,i_category 
+      ,i_class 
+      ,i_current_price
+      ,sum(ss_ext_sales_price) as itemrevenue 
+      ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over
+          (partition by i_class) as revenueratio
+from	
+	store_sales
+    	,item 
+    	,date_dim
+where 
+	ss_item_sk = i_item_sk 
+  	and i_category in ('Jewelry', 'Sports', 'Books')
+  	and ss_sold_date_sk = d_date_sk
+	and d_date between cast('2001-01-12' as date) 
+				and (cast('2001-01-12' as date) + 30 days)
+group by 
+	i_item_id
+        ,i_item_desc 
+        ,i_category
+        ,i_class
+        ,i_current_price
+order by 
+	i_category
+        ,i_class
+        ,i_item_id
+        ,i_item_desc
+        ,revenueratio;
+
+-- end query 1 in stream 0 using template query98.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query99.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query99.q b/ql/src/test/queries/clientpositive/perf/query99.q
new file mode 100644
index 0000000..83be1d0
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query99.q
@@ -0,0 +1,37 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query99.tpl and seed 1819994127
+explain
+select  
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,cc_name
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end)  as `30 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and 
+                 (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end )  as `31-60 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and 
+                 (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end)  as `61-90 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and
+                 (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end)  as `91-120 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk  > 120) then 1 else 0 end)  as `>120 days` 
+from
+   catalog_sales
+  ,warehouse
+  ,ship_mode
+  ,call_center
+  ,date_dim
+where
+    d_month_seq between 1212 and 1212 + 11
+and cs_ship_date_sk   = d_date_sk
+and cs_warehouse_sk   = w_warehouse_sk
+and cs_ship_mode_sk   = sm_ship_mode_sk
+and cs_call_center_sk = cc_call_center_sk
+group by
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,cc_name
+order by substr(w_warehouse_name,1,20)
+        ,sm_type
+        ,cc_name
+limit 100;
+
+-- end query 1 in stream 0 using template query99.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query1.q.out b/ql/src/test/results/clientpositive/perf/query1.q.out
index 7c583e4..da4a65c 100644
--- a/ql/src/test/results/clientpositive/perf/query1.q.out
+++ b/ql/src/test/results/clientpositive/perf/query1.q.out
@@ -1,4 +1,5 @@
-PREHOOK: query: explain with customer_total_return as
+PREHOOK: query: explain
+with customer_total_return as
 (select sr_customer_sk as ctr_customer_sk
 ,sr_store_sk as ctr_store_sk
 ,sum(SR_FEE) as ctr_total_return
@@ -21,7 +22,8 @@ and ctr1.ctr_customer_sk = c_customer_sk
 order by c_customer_id
 limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with customer_total_return as
+POSTHOOK: query: explain
+with customer_total_return as
 (select sr_customer_sk as ctr_customer_sk
 ,sr_store_sk as ctr_store_sk
 ,sum(SR_FEE) as ctr_total_return

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query10.q.out b/ql/src/test/results/clientpositive/perf/query10.q.out
new file mode 100644
index 0000000..9b6621c
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query10.q.out
@@ -0,0 +1,296 @@
+PREHOOK: query: explain
+select  
+  cd_gender,
+  cd_marital_status,
+  cd_education_status,
+  count(*) cnt1,
+  cd_purchase_estimate,
+  count(*) cnt2,
+  cd_credit_rating,
+  count(*) cnt3,
+  cd_dep_count,
+  count(*) cnt4,
+  cd_dep_employed_count,
+  count(*) cnt5,
+  cd_dep_college_count,
+  count(*) cnt6
+ from
+  customer c,customer_address ca,customer_demographics
+ where
+  c.c_current_addr_sk = ca.ca_address_sk and
+  ca_county in ('Walker County','Richland County','Gaines County','Douglas County','Dona Ana County') and
+  cd_demo_sk = c.c_current_cdemo_sk and 
+  exists (select *
+          from store_sales,date_dim
+          where c.c_customer_sk = ss_customer_sk and
+                ss_sold_date_sk = d_date_sk and
+                d_year = 2002 and
+                d_moy between 4 and 4+3) and
+   (exists (select *
+            from web_sales,date_dim
+            where c.c_customer_sk = ws_bill_customer_sk and
+                  ws_sold_date_sk = d_date_sk and
+                  d_year = 2002 and
+                  d_moy between 4 ANd 4+3) or 
+    exists (select * 
+            from catalog_sales,date_dim
+            where c.c_customer_sk = cs_ship_customer_sk and
+                  cs_sold_date_sk = d_date_sk and
+                  d_year = 2002 and
+                  d_moy between 4 and 4+3))
+ group by cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ order by cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select  
+  cd_gender,
+  cd_marital_status,
+  cd_education_status,
+  count(*) cnt1,
+  cd_purchase_estimate,
+  count(*) cnt2,
+  cd_credit_rating,
+  count(*) cnt3,
+  cd_dep_count,
+  count(*) cnt4,
+  cd_dep_employed_count,
+  count(*) cnt5,
+  cd_dep_college_count,
+  count(*) cnt6
+ from
+  customer c,customer_address ca,customer_demographics
+ where
+  c.c_current_addr_sk = ca.ca_address_sk and
+  ca_county in ('Walker County','Richland County','Gaines County','Douglas County','Dona Ana County') and
+  cd_demo_sk = c.c_current_cdemo_sk and 
+  exists (select *
+          from store_sales,date_dim
+          where c.c_customer_sk = ss_customer_sk and
+                ss_sold_date_sk = d_date_sk and
+                d_year = 2002 and
+                d_moy between 4 and 4+3) and
+   (exists (select *
+            from web_sales,date_dim
+            where c.c_customer_sk = ws_bill_customer_sk and
+                  ws_sold_date_sk = d_date_sk and
+                  d_year = 2002 and
+                  d_moy between 4 ANd 4+3) or 
+    exists (select * 
+            from catalog_sales,date_dim
+            where c.c_customer_sk = cs_ship_customer_sk and
+                  cs_sold_date_sk = d_date_sk and
+                  d_year = 2002 and
+                  d_moy between 4 and 4+3))
+ group by cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ order by cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 11 <- Map 10 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE)
+Reducer 12 <- Reducer 11 (SIMPLE_EDGE)
+Reducer 14 <- Map 13 (SIMPLE_EDGE), Map 18 (SIMPLE_EDGE)
+Reducer 15 <- Reducer 14 (SIMPLE_EDGE)
+Reducer 16 <- Map 13 (SIMPLE_EDGE), Map 19 (SIMPLE_EDGE)
+Reducer 17 <- Reducer 16 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 3 <- Map 9 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 15 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 17 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 7
+      File Output Operator [FS_74]
+        Limit [LIM_73] (rows=100 width=88)
+          Number of rows:100
+          Select Operator [SEL_72] (rows=383325119 width=88)
+            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13"]
+          <-Reducer 6 [SIMPLE_EDGE]
+            SHUFFLE [RS_71]
+              Select Operator [SEL_70] (rows=383325119 width=88)
+                Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col8","_col10","_col12"]
+                Group By Operator [GBY_69] (rows=383325119 width=88)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7
+                <-Reducer 5 [SIMPLE_EDGE]
+                  SHUFFLE [RS_68]
+                    PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+                    Group By Operator [GBY_67] (rows=766650239 width=88)
+                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["count()"],keys:_col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
+                      Select Operator [SEL_66] (rows=766650239 width=88)
+                        Output:["_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13"]
+                        Filter Operator [FIL_65] (rows=766650239 width=88)
+                          predicate:(_col16 is not null or _col18 is not null)
+                          Merge Join Operator [MERGEJOIN_112] (rows=766650239 width=88)
+                            Conds:RS_62._col0=RS_63._col0(Left Outer),Output:["_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col16","_col18"]
+                          <-Reducer 17 [SIMPLE_EDGE]
+                            SHUFFLE [RS_63]
+                              PartitionCols:_col0
+                              Select Operator [SEL_61] (rows=158394413 width=135)
+                                Output:["_col0","_col1"]
+                                Group By Operator [GBY_60] (rows=158394413 width=135)
+                                  Output:["_col0"],keys:KEY._col0
+                                <-Reducer 16 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_59]
+                                    PartitionCols:_col0
+                                    Group By Operator [GBY_58] (rows=316788826 width=135)
+                                      Output:["_col0"],keys:_col1
+                                      Merge Join Operator [MERGEJOIN_110] (rows=316788826 width=135)
+                                        Conds:RS_54._col0=RS_55._col0(Inner),Output:["_col1"]
+                                      <-Map 13 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_55]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_53] (rows=4058 width=1119)
+                                            Output:["_col0"]
+                                            Filter Operator [FIL_105] (rows=4058 width=1119)
+                                              predicate:((d_year = 2002) and d_moy BETWEEN 4 AND 7 and d_date_sk is not null)
+                                              TableScan [TS_12] (rows=73049 width=1119)
+                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
+                                      <-Map 19 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_54]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_50] (rows=287989836 width=135)
+                                            Output:["_col0","_col1"]
+                                            Filter Operator [FIL_104] (rows=287989836 width=135)
+                                              predicate:(cs_ship_customer_sk is not null and cs_sold_date_sk is not null)
+                                              TableScan [TS_48] (rows=287989836 width=135)
+                                                default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_ship_customer_sk"]
+                          <-Reducer 4 [SIMPLE_EDGE]
+                            SHUFFLE [RS_62]
+                              PartitionCols:_col0
+                              Select Operator [SEL_47] (rows=696954748 width=88)
+                                Output:["_col0","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                Merge Join Operator [MERGEJOIN_111] (rows=696954748 width=88)
+                                  Conds:RS_43._col0=RS_44._col0(Left Outer),RS_43._col0=RS_45._col0(Inner),Output:["_col0","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col15"]
+                                <-Reducer 12 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_44]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_22] (rows=79201469 width=135)
+                                      Output:["_col0","_col1"]
+                                      Group By Operator [GBY_21] (rows=79201469 width=135)
+                                        Output:["_col0"],keys:KEY._col0
+                                      <-Reducer 11 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_20]
+                                          PartitionCols:_col0
+                                          Group By Operator [GBY_19] (rows=158402938 width=135)
+                                            Output:["_col0"],keys:_col1
+                                            Merge Join Operator [MERGEJOIN_108] (rows=158402938 width=135)
+                                              Conds:RS_15._col0=RS_16._col0(Inner),Output:["_col1"]
+                                            <-Map 13 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_16]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_14] (rows=4058 width=1119)
+                                                  Output:["_col0"]
+                                                  Filter Operator [FIL_101] (rows=4058 width=1119)
+                                                    predicate:((d_year = 2002) and d_moy BETWEEN 4 AND 7 and d_date_sk is not null)
+                                                     Please refer to the previous TableScan [TS_12]
+                                            <-Map 10 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_15]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_11] (rows=144002668 width=135)
+                                                  Output:["_col0","_col1"]
+                                                  Filter Operator [FIL_100] (rows=144002668 width=135)
+                                                    predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null)
+                                                    TableScan [TS_9] (rows=144002668 width=135)
+                                                      default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_bill_customer_sk"]
+                                <-Reducer 15 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_45]
+                                    PartitionCols:_col0
+                                    Group By Operator [GBY_35] (rows=316797606 width=88)
+                                      Output:["_col0"],keys:KEY._col0
+                                    <-Reducer 14 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_34]
+                                        PartitionCols:_col0
+                                        Group By Operator [GBY_33] (rows=633595212 width=88)
+                                          Output:["_col0"],keys:_col1
+                                          Merge Join Operator [MERGEJOIN_109] (rows=633595212 width=88)
+                                            Conds:RS_29._col0=RS_30._col0(Inner),Output:["_col1"]
+                                          <-Map 13 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_30]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_28] (rows=4058 width=1119)
+                                                Output:["_col0"]
+                                                Filter Operator [FIL_103] (rows=4058 width=1119)
+                                                  predicate:((d_year = 2002) and d_moy BETWEEN 4 AND 7 and d_date_sk is not null)
+                                                   Please refer to the previous TableScan [TS_12]
+                                          <-Map 18 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_29]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_25] (rows=575995635 width=88)
+                                                Output:["_col0","_col1"]
+                                                Filter Operator [FIL_102] (rows=575995635 width=88)
+                                                  predicate:(ss_customer_sk is not null and ss_sold_date_sk is not null)
+                                                  TableScan [TS_23] (rows=575995635 width=88)
+                                                    default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_customer_sk"]
+                                <-Reducer 3 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_43]
+                                    PartitionCols:_col0
+                                    Merge Join Operator [MERGEJOIN_107] (rows=96800003 width=860)
+                                      Conds:RS_40._col1=RS_41._col0(Inner),Output:["_col0","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13"]
+                                    <-Map 9 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_41]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_8] (rows=1861800 width=385)
+                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
+                                          Filter Operator [FIL_99] (rows=1861800 width=385)
+                                            predicate:cd_demo_sk is not null
+                                            TableScan [TS_6] (rows=1861800 width=385)
+                                              default@customer_demographics,customer_demographics,Tbl:COMPLETE,Col:NONE,Output:["cd_demo_sk","cd_gender","cd_marital_status","cd_education_status","cd_purchase_estimate","cd_credit_rating","cd_dep_count","cd_dep_employed_count","cd_dep_college_count"]
+                                    <-Reducer 2 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_40]
+                                        PartitionCols:_col1
+                                        Merge Join Operator [MERGEJOIN_106] (rows=88000001 width=860)
+                                          Conds:RS_37._col2=RS_38._col0(Inner),Output:["_col0","_col1"]
+                                        <-Map 1 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_37]
+                                            PartitionCols:_col2
+                                            Select Operator [SEL_2] (rows=80000000 width=860)
+                                              Output:["_col0","_col1","_col2"]
+                                              Filter Operator [FIL_97] (rows=80000000 width=860)
+                                                predicate:(c_current_addr_sk is not null and c_current_cdemo_sk is not null)
+                                                TableScan [TS_0] (rows=80000000 width=860)
+                                                  default@customer,c,Tbl:COMPLETE,Col:NONE,Output:["c_customer_sk","c_current_cdemo_sk","c_current_addr_sk"]
+                                        <-Map 8 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_38]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_5] (rows=20000000 width=1014)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_98] (rows=20000000 width=1014)
+                                                predicate:((ca_county) IN ('Walker County', 'Richland County', 'Gaines County', 'Douglas County', 'Dona Ana County') and ca_address_sk is not null)
+                                                TableScan [TS_3] (rows=40000000 width=1014)
+                                                  default@customer_address,ca,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"]
+

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query12.q.out b/ql/src/test/results/clientpositive/perf/query12.q.out
index fb6d1af..0506eca 100644
--- a/ql/src/test/results/clientpositive/perf/query12.q.out
+++ b/ql/src/test/results/clientpositive/perf/query12.q.out
@@ -1,27 +1,69 @@
-PREHOOK: query: explain 
-select 
-i_item_desc ,i_category ,i_class ,i_current_price ,i_item_id ,sum(ws_ext_sales_price) as itemrevenue ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over (partition by i_class) as revenueratio 
-from web_sales ,item ,date_dim 
+PREHOOK: query: explain
+select  i_item_desc 
+      ,i_category 
+      ,i_class 
+      ,i_current_price
+      ,sum(ws_ext_sales_price) as itemrevenue 
+      ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over
+          (partition by i_class) as revenueratio
+from	
+	web_sales
+    	,item 
+    	,date_dim
 where 
-web_sales.ws_item_sk = item.i_item_sk 
-and item.i_category in ('Jewelry', 'Sports', 'Books') 
-and web_sales.ws_sold_date_sk = date_dim.d_date_sk 
-and date_dim.d_date between cast('2001-01-12' as date)
-                                and (cast('2001-01-12' as date) + 30 days)
-group by i_item_id ,i_item_desc ,i_category ,i_class ,i_current_price order by i_category ,i_class ,i_item_id ,i_item_desc ,revenueratio limit 100
+	ws_item_sk = i_item_sk 
+  	and i_category in ('Jewelry', 'Sports', 'Books')
+  	and ws_sold_date_sk = d_date_sk
+	and d_date between cast('2001-01-12' as date) 
+				and (cast('2001-01-12' as date) + 30 days)
+group by 
+	i_item_id
+        ,i_item_desc 
+        ,i_category
+        ,i_class
+        ,i_current_price
+order by 
+	i_category
+        ,i_class
+        ,i_item_id
+        ,i_item_desc
+        ,revenueratio
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain 
-select 
-i_item_desc ,i_category ,i_class ,i_current_price ,i_item_id ,sum(ws_ext_sales_price) as itemrevenue ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over (partition by i_class) as revenueratio 
-from web_sales ,item ,date_dim 
+POSTHOOK: query: explain
+select  i_item_desc 
+      ,i_category 
+      ,i_class 
+      ,i_current_price
+      ,sum(ws_ext_sales_price) as itemrevenue 
+      ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over
+          (partition by i_class) as revenueratio
+from	
+	web_sales
+    	,item 
+    	,date_dim
 where 
-web_sales.ws_item_sk = item.i_item_sk 
-and item.i_category in ('Jewelry', 'Sports', 'Books') 
-and web_sales.ws_sold_date_sk = date_dim.d_date_sk 
-and date_dim.d_date between cast('2001-01-12' as date)
-                                and (cast('2001-01-12' as date) + 30 days)
-group by i_item_id ,i_item_desc ,i_category ,i_class ,i_current_price order by i_category ,i_class ,i_item_id ,i_item_desc ,revenueratio limit 100
+	ws_item_sk = i_item_sk 
+  	and i_category in ('Jewelry', 'Sports', 'Books')
+  	and ws_sold_date_sk = d_date_sk
+	and d_date between cast('2001-01-12' as date) 
+				and (cast('2001-01-12' as date) + 30 days)
+group by 
+	i_item_id
+        ,i_item_desc 
+        ,i_category
+        ,i_class
+        ,i_current_price
+order by 
+	i_category
+        ,i_class
+        ,i_item_id
+        ,i_item_desc
+        ,revenueratio
+limit 100
 POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
 Vertex dependency in root stage
 Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
 Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
@@ -31,62 +73,66 @@ Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
       Reducer 6
-      File Output Operator [FS_26]
-        Limit [LIM_25] (rows=100 width=135)
+      File Output Operator [FS_29]
+        Limit [LIM_27] (rows=100 width=135)
           Number of rows:100
-          Select Operator [SEL_24] (rows=21780404 width=135)
-            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+          Select Operator [SEL_26] (rows=87121617 width=135)
+            Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           <-Reducer 5 [SIMPLE_EDGE]
-            SHUFFLE [RS_23]
-              Select Operator [SEL_21] (rows=21780404 width=135)
+            SHUFFLE [RS_25]
+              Select Operator [SEL_23] (rows=87121617 width=135)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                PTF Operator [PTF_20] (rows=21780404 width=135)
+                PTF Operator [PTF_22] (rows=87121617 width=135)
                   Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3 ASC NULLS FIRST","partition by:":"_col3"}]
-                  Select Operator [SEL_19] (rows=21780404 width=135)
+                  Select Operator [SEL_21] (rows=87121617 width=135)
                     Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                   <-Reducer 4 [SIMPLE_EDGE]
-                    SHUFFLE [RS_18]
+                    SHUFFLE [RS_20]
                       PartitionCols:_col3
-                      Group By Operator [GBY_17] (rows=21780404 width=135)
-                        Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
-                      <-Reducer 3 [SIMPLE_EDGE]
-                        SHUFFLE [RS_16]
-                          PartitionCols:_col0, _col1, _col2, _col3, _col4
-                          Group By Operator [GBY_15] (rows=43560808 width=135)
-                            Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col23)"],keys:_col38, _col41, _col49, _col47, _col42
-                            Select Operator [SEL_14] (rows=43560808 width=135)
-                              Output:["_col23","_col38","_col41","_col42","_col47","_col49"]
-                              Filter Operator [FIL_32] (rows=43560808 width=135)
-                                predicate:((_col3 = _col37) and (_col0 = _col62))
-                                Merge Join Operator [MERGEJOIN_37] (rows=174243235 width=135)
-                                  Conds:RS_9._col0=RS_11.d_date_sk(Inner),Output:["_col0","_col3","_col23","_col37","_col38","_col41","_col42","_col47","_col49","_col62"]
-                                <-Map 8 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_11]
-                                    PartitionCols:d_date_sk
-                                    Filter Operator [FIL_35] (rows=8116 width=1119)
-                                      predicate:(d_date_sk is not null and d_date BETWEEN 2001-01-12 AND 2001-02-11 00:00:00.0)
-                                      TableScan [TS_2] (rows=73049 width=1119)
-                                        default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
-                                <-Reducer 2 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_9]
-                                    PartitionCols:_col0
-                                    Merge Join Operator [MERGEJOIN_36] (rows=158402938 width=135)
-                                      Conds:RS_4.ws_item_sk=RS_6.i_item_sk(Inner),Output:["_col0","_col3","_col23","_col37","_col38","_col41","_col42","_col47","_col49"]
-                                    <-Map 1 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_4]
-                                        PartitionCols:ws_item_sk
-                                        Filter Operator [FIL_33] (rows=144002668 width=135)
+                      Select Operator [SEL_19] (rows=87121617 width=135)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                        Group By Operator [GBY_18] (rows=87121617 width=135)
+                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
+                        <-Reducer 3 [SIMPLE_EDGE]
+                          SHUFFLE [RS_17]
+                            PartitionCols:_col0, _col1, _col2, _col3, _col4
+                            Group By Operator [GBY_16] (rows=174243235 width=135)
+                              Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col2)"],keys:_col10, _col9, _col6, _col7, _col8
+                              Merge Join Operator [MERGEJOIN_39] (rows=174243235 width=135)
+                                Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col2","_col6","_col7","_col8","_col9","_col10"]
+                              <-Map 8 [SIMPLE_EDGE]
+                                SHUFFLE [RS_13]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_8] (rows=231000 width=1436)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                    Filter Operator [FIL_37] (rows=231000 width=1436)
+                                      predicate:((i_category) IN ('Jewelry', 'Sports', 'Books') and i_item_sk is not null)
+                                      TableScan [TS_6] (rows=462000 width=1436)
+                                        default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_item_id","i_item_desc","i_current_price","i_class","i_category"]
+                              <-Reducer 2 [SIMPLE_EDGE]
+                                SHUFFLE [RS_12]
+                                  PartitionCols:_col1
+                                  Merge Join Operator [MERGEJOIN_38] (rows=158402938 width=135)
+                                    Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col1","_col2"]
+                                  <-Map 1 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_9]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_2] (rows=144002668 width=135)
+                                        Output:["_col0","_col1","_col2"]
+                                        Filter Operator [FIL_35] (rows=144002668 width=135)
                                           predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
                                           TableScan [TS_0] (rows=144002668 width=135)
                                             default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_ext_sales_price"]
-                                    <-Map 7 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_6]
-                                        PartitionCols:i_item_sk
-                                        Filter Operator [FIL_34] (rows=231000 width=1436)
-                                          predicate:(i_item_sk is not null and (i_category) IN ('Jewelry', 'Sports', 'Books'))
-                                          TableScan [TS_1] (rows=462000 width=1436)
-                                            default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_item_id","i_item_desc","i_current_price","i_class","i_category"]
+                                  <-Map 7 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_10]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_5] (rows=8116 width=1119)
+                                        Output:["_col0"]
+                                        Filter Operator [FIL_36] (rows=8116 width=1119)
+                                          predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 2001-01-12 00:00:00.0 AND 2001-02-11 00:00:00.0 and d_date_sk is not null)
+                                          TableScan [TS_3] (rows=73049 width=1119)
+                                            default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
 


[15/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query48.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query48.q b/ql/src/test/queries/clientpositive/perf/query48.q
index a02f630..cfff1d7 100644
--- a/ql/src/test/queries/clientpositive/perf/query48.q
+++ b/ql/src/test/queries/clientpositive/perf/query48.q
@@ -1,2 +1,69 @@
 set hive.mapred.mode=nonstrict;
-explain select sum (ss_quantity) from store_sales, store, customer_demographics, customer_address, date_dim where store.s_store_sk = store_sales.ss_store_sk and store_sales.ss_sold_date_sk = date_dim.d_date_sk and d_year = 1998 and ( ( customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and ss_sales_price between 100.00 and 150.00 ) or ( customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and ss_sales_price between 50.00 and 100.00 ) or ( customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and ss_sales_price between 150.00 and 200.00 ) ) and ( ( store_sales.ss_addr_sk = customer_address.ca_address_sk and ca_country = 'United States' and ca_state in ('KY', 'GA', 'NM') and ss_net_profit between 0 and 2000 ) or (store_sales.ss_addr_sk = customer_address.ca_address_
 sk and ca_country = 'United States' and ca_state in ('MT', 'OR', 'IN') and ss_net_profit between 150 and 3000 ) or (store_sales.ss_addr_sk = customer_address.ca_address_sk and ca_country = 'United States' and ca_state in ('WI', 'MO', 'WV') and ss_net_profit between 50 and 25000 ) ) ;
+-- start query 1 in stream 0 using template query48.tpl and seed 622697896
+explain
+select sum (ss_quantity)
+ from store_sales, store, customer_demographics, customer_address, date_dim
+ where s_store_sk = ss_store_sk
+ and  ss_sold_date_sk = d_date_sk and d_year = 1998
+ and  
+ (
+  (
+   cd_demo_sk = ss_cdemo_sk
+   and 
+   cd_marital_status = 'M'
+   and 
+   cd_education_status = '4 yr Degree'
+   and 
+   ss_sales_price between 100.00 and 150.00  
+   )
+ or
+  (
+  cd_demo_sk = ss_cdemo_sk
+   and 
+   cd_marital_status = 'M'
+   and 
+   cd_education_status = '4 yr Degree'
+   and 
+   ss_sales_price between 50.00 and 100.00   
+  )
+ or 
+ (
+  cd_demo_sk = ss_cdemo_sk
+  and 
+   cd_marital_status = 'M'
+   and 
+   cd_education_status = '4 yr Degree'
+   and 
+   ss_sales_price between 150.00 and 200.00  
+ )
+ )
+ and
+ (
+  (
+  ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('KY', 'GA', 'NM')
+  and ss_net_profit between 0 and 2000  
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('MT', 'OR', 'IN')
+  and ss_net_profit between 150 and 3000 
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('WI', 'MO', 'WV')
+  and ss_net_profit between 50 and 25000 
+  )
+ )
+;
+
+-- end query 1 in stream 0 using template query48.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query49.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query49.q b/ql/src/test/queries/clientpositive/perf/query49.q
new file mode 100644
index 0000000..6c62e1f
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query49.q
@@ -0,0 +1,129 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query49.tpl and seed 1819994127
+explain
+select  
+ 'web' as channel
+ ,web.item
+ ,web.return_ratio
+ ,web.return_rank
+ ,web.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select ws.ws_item_sk as item
+ 		,(cast(sum(coalesce(wr.wr_return_quantity,0)) as dec(15,4))/
+ 		cast(sum(coalesce(ws.ws_quantity,0)) as dec(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(wr.wr_return_amt,0)) as dec(15,4))/
+ 		cast(sum(coalesce(ws.ws_net_paid,0)) as dec(15,4) )) as currency_ratio
+ 		from 
+ 		 web_sales ws left outer join web_returns wr 
+ 			on (ws.ws_order_number = wr.wr_order_number and 
+ 			ws.ws_item_sk = wr.wr_item_sk)
+                 ,date_dim
+ 		where 
+ 			wr.wr_return_amt > 10000 
+ 			and ws.ws_net_profit > 1
+                         and ws.ws_net_paid > 0
+                         and ws.ws_quantity > 0
+                         and ws_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+ 		group by ws.ws_item_sk
+ 	) in_web
+ ) web
+ where 
+ (
+ web.return_rank <= 10
+ or
+ web.currency_rank <= 10
+ )
+ union
+ select 
+ 'catalog' as channel
+ ,catalog.item
+ ,catalog.return_ratio
+ ,catalog.return_rank
+ ,catalog.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select 
+ 		cs.cs_item_sk as item
+ 		,(cast(sum(coalesce(cr.cr_return_quantity,0)) as dec(15,4))/
+ 		cast(sum(coalesce(cs.cs_quantity,0)) as dec(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(cr.cr_return_amount,0)) as dec(15,4))/
+ 		cast(sum(coalesce(cs.cs_net_paid,0)) as dec(15,4) )) as currency_ratio
+ 		from 
+ 		catalog_sales cs left outer join catalog_returns cr
+ 			on (cs.cs_order_number = cr.cr_order_number and 
+ 			cs.cs_item_sk = cr.cr_item_sk)
+                ,date_dim
+ 		where 
+ 			cr.cr_return_amount > 10000 
+ 			and cs.cs_net_profit > 1
+                         and cs.cs_net_paid > 0
+                         and cs.cs_quantity > 0
+                         and cs_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+                 group by cs.cs_item_sk
+ 	) in_cat
+ ) catalog
+ where 
+ (
+ catalog.return_rank <= 10
+ or
+ catalog.currency_rank <=10
+ )
+ union
+ select 
+ 'store' as channel
+ ,store.item
+ ,store.return_ratio
+ ,store.return_rank
+ ,store.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select sts.ss_item_sk as item
+ 		,(cast(sum(coalesce(sr.sr_return_quantity,0)) as dec(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as dec(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(sr.sr_return_amt,0)) as dec(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as dec(15,4) )) as currency_ratio
+ 		from 
+ 		store_sales sts left outer join store_returns sr
+ 			on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk)
+                ,date_dim
+ 		where 
+ 			sr.sr_return_amt > 10000 
+ 			and sts.ss_net_profit > 1
+                         and sts.ss_net_paid > 0 
+                         and sts.ss_quantity > 0
+                         and ss_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+ 		group by sts.ss_item_sk
+ 	) in_store
+ ) store
+ where  (
+ store.return_rank <= 10
+ or 
+ store.currency_rank <= 10
+ )
+ order by 1,4,5
+ limit 100;
+
+-- end query 1 in stream 0 using template query49.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query5.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query5.q b/ql/src/test/queries/clientpositive/perf/query5.q
index a877c4c..bf61fb2 100644
--- a/ql/src/test/queries/clientpositive/perf/query5.q
+++ b/ql/src/test/queries/clientpositive/perf/query5.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query5.tpl and seed 1819994127
 explain
 with ssr as
  (select s_store_id,
@@ -125,3 +127,4 @@ with ssr as
          ,id
  limit 100;
 
+-- end query 1 in stream 0 using template query5.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query50.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query50.q b/ql/src/test/queries/clientpositive/perf/query50.q
index e4bdf0a..0e2caf6 100644
--- a/ql/src/test/queries/clientpositive/perf/query50.q
+++ b/ql/src/test/queries/clientpositive/perf/query50.q
@@ -1,4 +1,6 @@
-explain 
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query50.tpl and seed 1819994127
+explain
 select  
    s_store_name
   ,s_company_id
@@ -10,14 +12,14 @@ select
   ,s_county
   ,s_state
   ,s_zip
-  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end)  as 30days 
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end)  as `30 days` 
   ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and 
-                 (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end )  as 3160days 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end )  as `31-60 days` 
   ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and 
-                 (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end)  as 6190days 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end)  as `61-90 days` 
   ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and
-                 (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end)  as 91120days 
-  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk  > 120) then 1 else 0 end)  as 120days 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end)  as `91-120 days` 
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk  > 120) then 1 else 0 end)  as `>120 days` 
 from
    store_sales
   ,store_returns
@@ -27,12 +29,12 @@ from
 where
     d2.d_year = 2000
 and d2.d_moy  = 9
-and store_sales.ss_ticket_number = store_returns.sr_ticket_number
-and store_sales.ss_item_sk = store_returns.sr_item_sk
-and store_sales.ss_sold_date_sk   = d1.d_date_sk
+and ss_ticket_number = sr_ticket_number
+and ss_item_sk = sr_item_sk
+and ss_sold_date_sk   = d1.d_date_sk
 and sr_returned_date_sk   = d2.d_date_sk
-and store_sales.ss_customer_sk = store_returns.sr_customer_sk
-and store_sales.ss_store_sk = store.s_store_sk
+and ss_customer_sk = sr_customer_sk
+and ss_store_sk = s_store_sk
 group by
    s_store_name
   ,s_company_id
@@ -56,3 +58,4 @@ order by s_store_name
         ,s_zip
 limit 100;
 
+-- end query 1 in stream 0 using template query50.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query51.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query51.q b/ql/src/test/queries/clientpositive/perf/query51.q
index b7688cf..9f90525 100644
--- a/ql/src/test/queries/clientpositive/perf/query51.q
+++ b/ql/src/test/queries/clientpositive/perf/query51.q
@@ -1,24 +1,26 @@
 set hive.mapred.mode=nonstrict;
-explain WITH web_v1 as (
+-- start query 1 in stream 0 using template query51.tpl and seed 1819994127
+explain
+WITH web_v1 as (
 select
-  ws_item_sk item_sk, d_date, sum(ws_sales_price),
+  ws_item_sk item_sk, d_date,
   sum(sum(ws_sales_price))
       over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
 from web_sales
     ,date_dim
 where ws_sold_date_sk=d_date_sk
-  and d_month_seq between 1193 and 1193+11
+  and d_month_seq between 1212 and 1212+11
   and ws_item_sk is not NULL
 group by ws_item_sk, d_date),
 store_v1 as (
 select
-  ss_item_sk item_sk, d_date, sum(ss_sales_price),
+  ss_item_sk item_sk, d_date,
   sum(sum(ss_sales_price))
       over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
 from store_sales
     ,date_dim
 where ss_sold_date_sk=d_date_sk
-  and d_month_seq between 1193 and 1193+11
+  and d_month_seq between 1212 and 1212+11
   and ss_item_sk is not NULL
 group by ss_item_sk, d_date)
  select  *
@@ -41,3 +43,5 @@ where web_cumulative > store_cumulative
 order by item_sk
         ,d_date
 limit 100;
+
+-- end query 1 in stream 0 using template query51.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query52.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query52.q b/ql/src/test/queries/clientpositive/perf/query52.q
index 24d0ac3..1fee846 100644
--- a/ql/src/test/queries/clientpositive/perf/query52.q
+++ b/ql/src/test/queries/clientpositive/perf/query52.q
@@ -1 +1,24 @@
-explain select dt.d_year ,item.i_brand_id brand_id ,item.i_brand brand ,sum(ss_ext_sales_price) ext_price from date_dim dt ,store_sales ,item where dt.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and item.i_manager_id = 1 and dt.d_moy=12 and dt.d_year=1998 group by dt.d_year ,item.i_brand ,item.i_brand_id order by dt.d_year ,ext_price desc ,brand_id limit 100 ;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query52.tpl and seed 1819994127
+explain
+select  dt.d_year
+ 	,item.i_brand_id brand_id
+ 	,item.i_brand brand
+ 	,sum(ss_ext_sales_price) ext_price
+ from date_dim dt
+     ,store_sales
+     ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+    and store_sales.ss_item_sk = item.i_item_sk
+    and item.i_manager_id = 1
+    and dt.d_moy=12
+    and dt.d_year=1998
+ group by dt.d_year
+ 	,item.i_brand
+ 	,item.i_brand_id
+ order by dt.d_year
+ 	,ext_price desc
+ 	,brand_id
+limit 100 ;
+
+-- end query 1 in stream 0 using template query52.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query53.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query53.q b/ql/src/test/queries/clientpositive/perf/query53.q
new file mode 100644
index 0000000..0b81574
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query53.q
@@ -0,0 +1,30 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query53.tpl and seed 1819994127
+explain
+select  * from 
+(select i_manufact_id,
+sum(ss_sales_price) sum_sales,
+avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales
+from item, store_sales, date_dim, store
+where ss_item_sk = i_item_sk and
+ss_sold_date_sk = d_date_sk and
+ss_store_sk = s_store_sk and
+d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11) and
+((i_category in ('Books','Children','Electronics') and
+i_class in ('personal','portable','reference','self-help') and
+i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+		'exportiunivamalg #9','scholaramalgamalg #9'))
+or(i_category in ('Women','Music','Men') and
+i_class in ('accessories','classical','fragrances','pants') and
+i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+		'importoamalg #1')))
+group by i_manufact_id, d_qoy ) tmp1
+where case when avg_quarterly_sales > 0 
+	then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales 
+	else null end > 0.1
+order by avg_quarterly_sales,
+	 sum_sales,
+	 i_manufact_id
+limit 100;
+
+-- end query 1 in stream 0 using template query53.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query54.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query54.q b/ql/src/test/queries/clientpositive/perf/query54.q
index b1a3e83..424f385 100644
--- a/ql/src/test/queries/clientpositive/perf/query54.q
+++ b/ql/src/test/queries/clientpositive/perf/query54.q
@@ -1 +1,58 @@
-explain with my_customers as ( select c_customer_sk , c_current_addr_sk from ( select cs_sold_date_sk sold_date_sk, cs_bill_customer_sk customer_sk, cs_item_sk item_sk from catalog_sales union all select ws_sold_date_sk sold_date_sk, ws_bill_customer_sk customer_sk, ws_item_sk item_sk from web_sales ) cs_or_ws_sales, item, date_dim, customer where sold_date_sk = d_date_sk and item_sk = i_item_sk and i_category = 'Jewelry' and i_class = 'football' and c_customer_sk = cs_or_ws_sales.customer_sk and d_moy = 3 and d_year = 2000 group by c_customer_sk , c_current_addr_sk ) , my_revenue as ( select c_customer_sk, sum(ss_ext_sales_price) as revenue from my_customers, store_sales, customer_address, store, date_dim where c_current_addr_sk = ca_address_sk and ca_county = s_county and ca_state = s_state and ss_sold_date_sk = d_date_sk and c_customer_sk = ss_customer_sk and d_month_seq between (1203) and (1205) group by c_customer_sk ) , segments as (select cast((revenue/50) as int) as segment 
 from my_revenue ) select segment, count(*) as num_customers, segment*50 as segment_base from segments group by segment order by segment, num_customers limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query54.tpl and seed 1930872976
+explain
+with my_customers as (
+ select distinct c_customer_sk
+        , c_current_addr_sk
+ from   
+        ( select cs_sold_date_sk sold_date_sk,
+                 cs_bill_customer_sk customer_sk,
+                 cs_item_sk item_sk
+          from   catalog_sales
+          union all
+          select ws_sold_date_sk sold_date_sk,
+                 ws_bill_customer_sk customer_sk,
+                 ws_item_sk item_sk
+          from   web_sales
+         ) cs_or_ws_sales,
+         item,
+         date_dim,
+         customer
+ where   sold_date_sk = d_date_sk
+         and item_sk = i_item_sk
+         and i_category = 'Jewelry'
+         and i_class = 'consignment'
+         and c_customer_sk = cs_or_ws_sales.customer_sk
+         and d_moy = 3
+         and d_year = 1999
+ )
+ , my_revenue as (
+ select c_customer_sk,
+        sum(ss_ext_sales_price) as revenue
+ from   my_customers,
+        store_sales,
+        customer_address,
+        store,
+        date_dim
+ where  c_current_addr_sk = ca_address_sk
+        and ca_county = s_county
+        and ca_state = s_state
+        and ss_sold_date_sk = d_date_sk
+        and c_customer_sk = ss_customer_sk
+        and d_month_seq between (select distinct d_month_seq+1
+                                 from   date_dim where d_year = 1999 and d_moy = 3)
+                           and  (select distinct d_month_seq+3
+                                 from   date_dim where d_year = 1999 and d_moy = 3)
+ group by c_customer_sk
+ )
+ , segments as
+ (select cast((revenue/50) as int) as segment
+  from   my_revenue
+ )
+  select  segment, count(*) as num_customers, segment*50 as segment_base
+ from segments
+ group by segment
+ order by segment, num_customers
+ limit 100;
+
+-- end query 1 in stream 0 using template query54.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query55.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query55.q b/ql/src/test/queries/clientpositive/perf/query55.q
index 644ba1a..f953f11 100644
--- a/ql/src/test/queries/clientpositive/perf/query55.q
+++ b/ql/src/test/queries/clientpositive/perf/query55.q
@@ -1 +1,16 @@
-explain select i_brand_id brand_id, i_brand brand, sum(ss_ext_sales_price) ext_price from date_dim, store_sales, item where date_dim.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and i_manager_id=36 and d_moy=12 and d_year=2001 group by i_brand, i_brand_id order by ext_price desc, i_brand_id limit 100 ;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query55.tpl and seed 2031708268
+explain
+select  i_brand_id brand_id, i_brand brand,
+ 	sum(ss_ext_sales_price) ext_price
+ from date_dim, store_sales, item
+ where d_date_sk = ss_sold_date_sk
+ 	and ss_item_sk = i_item_sk
+ 	and i_manager_id=36
+ 	and d_moy=12
+ 	and d_year=2001
+ group by i_brand, i_brand_id
+ order by ext_price desc, i_brand_id
+limit 100 ;
+
+-- end query 1 in stream 0 using template query55.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query56.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query56.q b/ql/src/test/queries/clientpositive/perf/query56.q
index 63e53ea..f3c8323 100644
--- a/ql/src/test/queries/clientpositive/perf/query56.q
+++ b/ql/src/test/queries/clientpositive/perf/query56.q
@@ -1,4 +1,7 @@
-explain with ss as (
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query56.tpl and seed 1951559352
+explain
+with ss as (
  select i_item_id,sum(ss_ext_sales_price) total_sales
  from
  	store_sales,
@@ -63,3 +66,5 @@ where i_color in ('orchid','chiffon','lace'))
  group by i_item_id
  order by total_sales
  limit 100;
+
+-- end query 1 in stream 0 using template query56.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query58.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query58.q b/ql/src/test/queries/clientpositive/perf/query58.q
index d392ded..8d918ef 100644
--- a/ql/src/test/queries/clientpositive/perf/query58.q
+++ b/ql/src/test/queries/clientpositive/perf/query58.q
@@ -1,4 +1,49 @@
-explain select  ss_items.item_id
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query58.tpl and seed 1819994127
+explain
+with ss_items as
+ (select i_item_id item_id
+        ,sum(ss_ext_sales_price) ss_item_rev 
+ from store_sales
+     ,item
+     ,date_dim
+ where ss_item_sk = i_item_sk
+   and d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq 
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+   and ss_sold_date_sk   = d_date_sk
+ group by i_item_id),
+ cs_items as
+ (select i_item_id item_id
+        ,sum(cs_ext_sales_price) cs_item_rev
+  from catalog_sales
+      ,item
+      ,date_dim
+ where cs_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq 
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+  and  cs_sold_date_sk = d_date_sk
+ group by i_item_id),
+ ws_items as
+ (select i_item_id item_id
+        ,sum(ws_ext_sales_price) ws_item_rev
+  from web_sales
+      ,item
+      ,date_dim
+ where ws_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq =(select d_week_seq 
+                                     from date_dim
+                                     where d_date = '1998-02-19'))
+  and ws_sold_date_sk   = d_date_sk
+ group by i_item_id)
+  select  ss_items.item_id
        ,ss_item_rev
        ,ss_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 ss_dev
        ,cs_item_rev
@@ -6,42 +51,17 @@ explain select  ss_items.item_id
        ,ws_item_rev
        ,ws_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 ws_dev
        ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average
-FROM
-( select i_item_id item_id ,sum(ss_ext_sales_price) as ss_item_rev 
- from store_sales
-     JOIN item ON store_sales.ss_item_sk = item.i_item_sk
-     JOIN date_dim ON store_sales.ss_sold_date_sk = date_dim.d_date_sk
-     JOIN (select d1.d_date
-                 from date_dim d1 JOIN date_dim d2 ON d1.d_week_seq = d2.d_week_seq
-                 where d2.d_date = '1998-08-04') sub ON date_dim.d_date = sub.d_date
- group by i_item_id ) ss_items
-JOIN
-( select i_item_id item_id ,sum(cs_ext_sales_price) as cs_item_rev 
- from catalog_sales
-     JOIN item ON catalog_sales.cs_item_sk = item.i_item_sk
-     JOIN date_dim ON catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
-     JOIN (select d1.d_date
-                 from date_dim d1 JOIN date_dim d2 ON d1.d_week_seq = d2.d_week_seq
-                 where d2.d_date = '1998-08-04') sub ON date_dim.d_date = sub.d_date
- group by i_item_id ) cs_items
-ON ss_items.item_id=cs_items.item_id
-JOIN
-( select i_item_id item_id ,sum(ws_ext_sales_price) as ws_item_rev 
- from web_sales
-     JOIN item ON web_sales.ws_item_sk = item.i_item_sk
-     JOIN date_dim ON web_sales.ws_sold_date_sk = date_dim.d_date_sk
-     JOIN (select d1.d_date
-                 from date_dim d1 JOIN date_dim d2 ON d1.d_week_seq = d2.d_week_seq
-                 where d2.d_date = '1998-08-04') sub ON date_dim.d_date = sub.d_date
- group by i_item_id ) ws_items
-ON ss_items.item_id=ws_items.item_id 
- where
-       ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+ from ss_items,cs_items,ws_items
+ where ss_items.item_id=cs_items.item_id
+   and ss_items.item_id=ws_items.item_id 
+   and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
    and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
    and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
    and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
    and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
    and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
- order by item_id ,ss_item_rev
+ order by item_id
+         ,ss_item_rev
  limit 100;
 
+-- end query 1 in stream 0 using template query58.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query59.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query59.q b/ql/src/test/queries/clientpositive/perf/query59.q
new file mode 100644
index 0000000..0999653
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query59.q
@@ -0,0 +1,46 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query59.tpl and seed 1819994127
+explain
+with wss as 
+ (select d_week_seq,
+        ss_store_sk,
+        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from store_sales,date_dim
+ where d_date_sk = ss_sold_date_sk
+ group by d_week_seq,ss_store_sk
+ )
+  select  s_store_name1,s_store_id1,d_week_seq1
+       ,sun_sales1/sun_sales2,mon_sales1/mon_sales2
+       ,tue_sales1/tue_sales1,wed_sales1/wed_sales2,thu_sales1/thu_sales2
+       ,fri_sales1/fri_sales2,sat_sales1/sat_sales2
+ from
+ (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1
+        ,s_store_id s_store_id1,sun_sales sun_sales1
+        ,mon_sales mon_sales1,tue_sales tue_sales1
+        ,wed_sales wed_sales1,thu_sales thu_sales1
+        ,fri_sales fri_sales1,sat_sales sat_sales1
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and 
+        d_month_seq between 1185 and 1185 + 11) y,
+ (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2
+        ,s_store_id s_store_id2,sun_sales sun_sales2
+        ,mon_sales mon_sales2,tue_sales tue_sales2
+        ,wed_sales wed_sales2,thu_sales thu_sales2
+        ,fri_sales fri_sales2,sat_sales sat_sales2
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and 
+        d_month_seq between 1185+ 12 and 1185 + 23) x
+ where s_store_id1=s_store_id2
+   and d_week_seq1=d_week_seq2-52
+ order by s_store_name1,s_store_id1,d_week_seq1
+limit 100;
+
+-- end query 1 in stream 0 using template query59.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query6.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query6.q b/ql/src/test/queries/clientpositive/perf/query6.q
index 8e4a7d3..d45045d 100644
--- a/ql/src/test/queries/clientpositive/perf/query6.q
+++ b/ql/src/test/queries/clientpositive/perf/query6.q
@@ -1,6 +1,7 @@
 set hive.mapred.mode=nonstrict;
-
-explain select  a.ca_state state, count(*) cnt
+-- start query 1 in stream 0 using template query6.tpl and seed 1819994127
+explain
+select  a.ca_state state, count(*) cnt
  from customer_address a
      ,customer c
      ,store_sales s
@@ -23,3 +24,5 @@ explain select  a.ca_state state, count(*) cnt
  having count(*) >= 10
  order by cnt 
  limit 100;
+
+-- end query 1 in stream 0 using template query6.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query60.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query60.q b/ql/src/test/queries/clientpositive/perf/query60.q
index efa86d1..a5ab248 100644
--- a/ql/src/test/queries/clientpositive/perf/query60.q
+++ b/ql/src/test/queries/clientpositive/perf/query60.q
@@ -1,5 +1,7 @@
+set hive.mapred.mode=nonstrict;
 -- start query 1 in stream 0 using template query60.tpl and seed 1930872976
-explain with ss as (
+explain
+with ss as (
  select
           i_item_id,sum(ss_ext_sales_price) total_sales
  from
@@ -75,3 +77,4 @@ where i_category in ('Children'))
       ,total_sales
  limit 100;
 
+-- end query 1 in stream 0 using template query60.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query61.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query61.q b/ql/src/test/queries/clientpositive/perf/query61.q
new file mode 100644
index 0000000..edaf6f6
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query61.q
@@ -0,0 +1,46 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query61.tpl and seed 1930872976
+explain
+select  promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100
+from
+  (select sum(ss_ext_sales_price) promotions
+   from  store_sales
+        ,store
+        ,promotion
+        ,date_dim
+        ,customer
+        ,customer_address 
+        ,item
+   where ss_sold_date_sk = d_date_sk
+   and   ss_store_sk = s_store_sk
+   and   ss_promo_sk = p_promo_sk
+   and   ss_customer_sk= c_customer_sk
+   and   ca_address_sk = c_current_addr_sk
+   and   ss_item_sk = i_item_sk 
+   and   ca_gmt_offset = -7
+   and   i_category = 'Electronics'
+   and   (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y')
+   and   s_gmt_offset = -7
+   and   d_year = 1999
+   and   d_moy  = 11) promotional_sales,
+  (select sum(ss_ext_sales_price) total
+   from  store_sales
+        ,store
+        ,date_dim
+        ,customer
+        ,customer_address
+        ,item
+   where ss_sold_date_sk = d_date_sk
+   and   ss_store_sk = s_store_sk
+   and   ss_customer_sk= c_customer_sk
+   and   ca_address_sk = c_current_addr_sk
+   and   ss_item_sk = i_item_sk
+   and   ca_gmt_offset = -7
+   and   i_category = 'Electronics'
+   and   s_gmt_offset = -7
+   and   d_year = 1999
+   and   d_moy  = 11) all_sales
+order by promotions, total
+limit 100;
+
+-- end query 1 in stream 0 using template query61.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query63.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query63.q b/ql/src/test/queries/clientpositive/perf/query63.q
new file mode 100644
index 0000000..49e513c
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query63.q
@@ -0,0 +1,31 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query63.tpl and seed 1819994127
+explain
+select  * 
+from (select i_manager_id
+             ,sum(ss_sales_price) sum_sales
+             ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales
+      from item
+          ,store_sales
+          ,date_dim
+          ,store
+      where ss_item_sk = i_item_sk
+        and ss_sold_date_sk = d_date_sk
+        and ss_store_sk = s_store_sk
+        and d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11)
+        and ((    i_category in ('Books','Children','Electronics')
+              and i_class in ('personal','portable','refernece','self-help')
+              and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+		                  'exportiunivamalg #9','scholaramalgamalg #9'))
+           or(    i_category in ('Women','Music','Men')
+              and i_class in ('accessories','classical','fragrances','pants')
+              and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+		                 'importoamalg #1')))
+group by i_manager_id, d_moy) tmp1
+where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+order by i_manager_id
+        ,avg_monthly_sales
+        ,sum_sales
+limit 100;
+
+-- end query 1 in stream 0 using template query63.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query64.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query64.q b/ql/src/test/queries/clientpositive/perf/query64.q
index 13ebf4a..b069c2a 100644
--- a/ql/src/test/queries/clientpositive/perf/query64.q
+++ b/ql/src/test/queries/clientpositive/perf/query64.q
@@ -1,2 +1,121 @@
 set hive.mapred.mode=nonstrict;
-explain select cs1.product_name ,cs1.store_name ,cs1.store_zip ,cs1.b_street_number ,cs1.b_streen_name ,cs1.b_city ,cs1.b_zip ,cs1.c_street_number ,cs1.c_street_name ,cs1.c_city ,cs1.c_zip ,cs1.syear ,cs1.cnt ,cs1.s1 ,cs1.s2 ,cs1.s3 ,cs2.s1 ,cs2.s2 ,cs2.s3 ,cs2.syear ,cs2.cnt from (select i_product_name as product_name ,i_item_sk as item_sk ,s_store_name as store_name ,s_zip as store_zip ,ad1.ca_street_number as b_street_number ,ad1.ca_street_name as b_streen_name ,ad1.ca_city as b_city ,ad1.ca_zip as b_zip ,ad2.ca_street_number as c_street_number ,ad2.ca_street_name as c_street_name ,ad2.ca_city as c_city ,ad2.ca_zip as c_zip ,d1.d_year as syear ,d2.d_year as fsyear ,d3.d_year as s2year ,count(*) as cnt ,sum(ss_wholesale_cost) as s1 ,sum(ss_list_price) as s2 ,sum(ss_coupon_amt) as s3 FROM store_sales JOIN store_returns ON store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number JOIN customer ON store_sales.ss_customer_sk = 
 customer.c_customer_sk JOIN date_dim d1 ON store_sales.ss_sold_date_sk = d1.d_date_sk JOIN date_dim d2 ON customer.c_first_sales_date_sk = d2.d_date_sk JOIN date_dim d3 ON customer.c_first_shipto_date_sk = d3.d_date_sk JOIN store ON store_sales.ss_store_sk = store.s_store_sk JOIN customer_demographics cd1 ON store_sales.ss_cdemo_sk= cd1.cd_demo_sk JOIN customer_demographics cd2 ON customer.c_current_cdemo_sk = cd2.cd_demo_sk JOIN promotion ON store_sales.ss_promo_sk = promotion.p_promo_sk JOIN household_demographics hd1 ON store_sales.ss_hdemo_sk = hd1.hd_demo_sk JOIN household_demographics hd2 ON customer.c_current_hdemo_sk = hd2.hd_demo_sk JOIN customer_address ad1 ON store_sales.ss_addr_sk = ad1.ca_address_sk JOIN customer_address ad2 ON customer.c_current_addr_sk = ad2.ca_address_sk JOIN income_band ib1 ON hd1.hd_income_band_sk = ib1.ib_income_band_sk JOIN income_band ib2 ON hd2.hd_income_band_sk = ib2.ib_income_band_sk JOIN item ON store_sales.ss_item_sk = item.i_item_sk JOIN (
 select cs_item_sk ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund from catalog_sales JOIN catalog_returns ON catalog_sales.cs_item_sk = catalog_returns.cr_item_sk and catalog_sales.cs_order_number = catalog_returns.cr_order_number group by cs_item_sk having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)) cs_ui ON store_sales.ss_item_sk = cs_ui.cs_item_sk WHERE cd1.cd_marital_status <> cd2.cd_marital_status and i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and i_current_price between 35 and 35 + 10 and i_current_price between 35 + 1 and 35 + 15 group by i_product_name ,i_item_sk ,s_store_name ,s_zip ,ad1.ca_street_number ,ad1.ca_street_name ,ad1.ca_city ,ad1.ca_zip ,ad2.ca_street_number ,ad2.ca_street_name ,ad2.ca_city ,ad2.ca_zip ,d1.d_year ,d2.d_year ,d3.d_year ) cs1 JOIN (select i_product_name as product_name ,i_item_sk as item_sk ,s_store_name as store_name ,s_zip as store
 _zip ,ad1.ca_street_number as b_street_number ,ad1.ca_street_name as b_streen_name ,ad1.ca_city as b_city ,ad1.ca_zip as b_zip ,ad2.ca_street_number as c_street_number ,ad2.ca_street_name as c_street_name ,ad2.ca_city as c_city ,ad2.ca_zip as c_zip ,d1.d_year as syear ,d2.d_year as fsyear ,d3.d_year as s2year ,count(*) as cnt ,sum(ss_wholesale_cost) as s1 ,sum(ss_list_price) as s2 ,sum(ss_coupon_amt) as s3 FROM store_sales JOIN store_returns ON store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number JOIN customer ON store_sales.ss_customer_sk = customer.c_customer_sk JOIN date_dim d1 ON store_sales.ss_sold_date_sk = d1.d_date_sk JOIN date_dim d2 ON customer.c_first_sales_date_sk = d2.d_date_sk JOIN date_dim d3 ON customer.c_first_shipto_date_sk = d3.d_date_sk JOIN store ON store_sales.ss_store_sk = store.s_store_sk JOIN customer_demographics cd1 ON store_sales.ss_cdemo_sk= cd1.cd_demo_sk JOIN customer_demographics cd2 ON cu
 stomer.c_current_cdemo_sk = cd2.cd_demo_sk JOIN promotion ON store_sales.ss_promo_sk = promotion.p_promo_sk JOIN household_demographics hd1 ON store_sales.ss_hdemo_sk = hd1.hd_demo_sk JOIN household_demographics hd2 ON customer.c_current_hdemo_sk = hd2.hd_demo_sk JOIN customer_address ad1 ON store_sales.ss_addr_sk = ad1.ca_address_sk JOIN customer_address ad2 ON customer.c_current_addr_sk = ad2.ca_address_sk JOIN income_band ib1 ON hd1.hd_income_band_sk = ib1.ib_income_band_sk JOIN income_band ib2 ON hd2.hd_income_band_sk = ib2.ib_income_band_sk JOIN item ON store_sales.ss_item_sk = item.i_item_sk JOIN (select cs_item_sk ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund from catalog_sales JOIN catalog_returns ON catalog_sales.cs_item_sk = catalog_returns.cr_item_sk and catalog_sales.cs_order_number = catalog_returns.cr_order_number group by cs_item_sk having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_cre
 dit)) cs_ui ON store_sales.ss_item_sk = cs_ui.cs_item_sk WHERE cd1.cd_marital_status <> cd2.cd_marital_status and i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and i_current_price between 35 and 35 + 10 and i_current_price between 35 + 1 and 35 + 15 group by i_product_name ,i_item_sk ,s_store_name ,s_zip ,ad1.ca_street_number ,ad1.ca_street_name ,ad1.ca_city ,ad1.ca_zip ,ad2.ca_street_number ,ad2.ca_street_name ,ad2.ca_city ,ad2.ca_zip ,d1.d_year ,d2.d_year ,d3.d_year ) cs2 ON cs1.item_sk=cs2.item_sk where cs1.syear = 2000 and cs2.syear = 2000 + 1 and cs2.cnt <= cs1.cnt and cs1.store_name = cs2.store_name and cs1.store_zip = cs2.store_zip order by cs1.product_name ,cs1.store_name ,cs2.cnt;
+-- start query 1 in stream 0 using template query64.tpl and seed 1220860970
+explain
+with cs_ui as
+ (select cs_item_sk
+        ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund
+  from catalog_sales
+      ,catalog_returns
+  where cs_item_sk = cr_item_sk
+    and cs_order_number = cr_order_number
+  group by cs_item_sk
+  having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)),
+cross_sales as
+ (select i_product_name product_name
+     ,i_item_sk item_sk
+     ,s_store_name store_name
+     ,s_zip store_zip
+     ,ad1.ca_street_number b_street_number
+     ,ad1.ca_street_name b_streen_name
+     ,ad1.ca_city b_city
+     ,ad1.ca_zip b_zip
+     ,ad2.ca_street_number c_street_number
+     ,ad2.ca_street_name c_street_name
+     ,ad2.ca_city c_city
+     ,ad2.ca_zip c_zip
+     ,d1.d_year as syear
+     ,d2.d_year as fsyear
+     ,d3.d_year s2year
+     ,count(*) cnt
+     ,sum(ss_wholesale_cost) s1
+     ,sum(ss_list_price) s2
+     ,sum(ss_coupon_amt) s3
+  FROM   store_sales
+        ,store_returns
+        ,cs_ui
+        ,date_dim d1
+        ,date_dim d2
+        ,date_dim d3
+        ,store
+        ,customer
+        ,customer_demographics cd1
+        ,customer_demographics cd2
+        ,promotion
+        ,household_demographics hd1
+        ,household_demographics hd2
+        ,customer_address ad1
+        ,customer_address ad2
+        ,income_band ib1
+        ,income_band ib2
+        ,item
+  WHERE  ss_store_sk = s_store_sk AND
+         ss_sold_date_sk = d1.d_date_sk AND
+         ss_customer_sk = c_customer_sk AND
+         ss_cdemo_sk= cd1.cd_demo_sk AND
+         ss_hdemo_sk = hd1.hd_demo_sk AND
+         ss_addr_sk = ad1.ca_address_sk and
+         ss_item_sk = i_item_sk and
+         ss_item_sk = sr_item_sk and
+         ss_ticket_number = sr_ticket_number and
+         ss_item_sk = cs_ui.cs_item_sk and
+         c_current_cdemo_sk = cd2.cd_demo_sk AND
+         c_current_hdemo_sk = hd2.hd_demo_sk AND
+         c_current_addr_sk = ad2.ca_address_sk and
+         c_first_sales_date_sk = d2.d_date_sk and
+         c_first_shipto_date_sk = d3.d_date_sk and
+         ss_promo_sk = p_promo_sk and
+         hd1.hd_income_band_sk = ib1.ib_income_band_sk and
+         hd2.hd_income_band_sk = ib2.ib_income_band_sk and
+         cd1.cd_marital_status <> cd2.cd_marital_status and
+         i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and
+         i_current_price between 35 and 35 + 10 and
+         i_current_price between 35 + 1 and 35 + 15
+group by i_product_name
+       ,i_item_sk
+       ,s_store_name
+       ,s_zip
+       ,ad1.ca_street_number
+       ,ad1.ca_street_name
+       ,ad1.ca_city
+       ,ad1.ca_zip
+       ,ad2.ca_street_number
+       ,ad2.ca_street_name
+       ,ad2.ca_city
+       ,ad2.ca_zip
+       ,d1.d_year
+       ,d2.d_year
+       ,d3.d_year
+)
+select cs1.product_name
+     ,cs1.store_name
+     ,cs1.store_zip
+     ,cs1.b_street_number
+     ,cs1.b_streen_name
+     ,cs1.b_city
+     ,cs1.b_zip
+     ,cs1.c_street_number
+     ,cs1.c_street_name
+     ,cs1.c_city
+     ,cs1.c_zip
+     ,cs1.syear
+     ,cs1.cnt
+     ,cs1.s1
+     ,cs1.s2
+     ,cs1.s3
+     ,cs2.s1
+     ,cs2.s2
+     ,cs2.s3
+     ,cs2.syear
+     ,cs2.cnt
+from cross_sales cs1,cross_sales cs2
+where cs1.item_sk=cs2.item_sk and
+     cs1.syear = 2000 and
+     cs2.syear = 2000 + 1 and
+     cs2.cnt <= cs1.cnt and
+     cs1.store_name = cs2.store_name and
+     cs1.store_zip = cs2.store_zip
+order by cs1.product_name
+       ,cs1.store_name
+       ,cs2.cnt;
+
+-- end query 1 in stream 0 using template query64.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query65.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query65.q b/ql/src/test/queries/clientpositive/perf/query65.q
index bd4475d..d5b53a2 100644
--- a/ql/src/test/queries/clientpositive/perf/query65.q
+++ b/ql/src/test/queries/clientpositive/perf/query65.q
@@ -1,38 +1,31 @@
 set hive.mapred.mode=nonstrict;
-explain select 
-    s_store_name,
-    i_item_desc,
-    sc.revenue,
-    i_current_price,
-    i_wholesale_cost,
-    i_brand
-from
-    store,
-    item,
-    (select 
-        ss_store_sk, avg(revenue) as ave
-    from
-        (select 
-        ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
-    from
-        store_sales, date_dim
-    where
-        ss_sold_date_sk = d_date_sk
-            and d_month_seq between 1212 and 1212 + 11
-    group by ss_store_sk , ss_item_sk) sa
-    group by ss_store_sk) sb,
-    (select 
-        ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
-    from
-        store_sales, date_dim
-    where
-        ss_sold_date_sk = d_date_sk
-            and d_month_seq between 1212 and 1212 + 11
-    group by ss_store_sk , ss_item_sk) sc
-where
-    sb.ss_store_sk = sc.ss_store_sk
-        and sc.revenue <= 0.1 * sb.ave
-        and s_store_sk = sc.ss_store_sk
-        and i_item_sk = sc.ss_item_sk
-order by s_store_name , i_item_desc
+-- start query 1 in stream 0 using template query65.tpl and seed 1819994127
+explain
+select 
+	s_store_name,
+	i_item_desc,
+	sc.revenue,
+	i_current_price,
+	i_wholesale_cost,
+	i_brand
+ from store, item,
+     (select ss_store_sk, avg(revenue) as ave
+ 	from
+ 	    (select  ss_store_sk, ss_item_sk, 
+ 		     sum(ss_sales_price) as revenue
+ 		from store_sales, date_dim
+ 		where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11
+ 		group by ss_store_sk, ss_item_sk) sa
+ 	group by ss_store_sk) sb,
+     (select  ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
+ 	from store_sales, date_dim
+ 	where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11
+ 	group by ss_store_sk, ss_item_sk) sc
+ where sb.ss_store_sk = sc.ss_store_sk and 
+       sc.revenue <= 0.1 * sb.ave and
+       s_store_sk = sc.ss_store_sk and
+       i_item_sk = sc.ss_item_sk
+ order by s_store_name, i_item_desc
 limit 100;
+
+-- end query 1 in stream 0 using template query65.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query66.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query66.q b/ql/src/test/queries/clientpositive/perf/query66.q
index 9224680..280bac8 100644
--- a/ql/src/test/queries/clientpositive/perf/query66.q
+++ b/ql/src/test/queries/clientpositive/perf/query66.q
@@ -1,11 +1,13 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query66.tpl and seed 2042478054
 explain
 select   
          w_warehouse_name
- 	 ,w_warehouse_sq_ft
- 	 ,w_city
- 	 ,w_county
- 	 ,w_state
- 	 ,w_country
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
         ,ship_carriers
         ,year
  	,sum(jan_sales) as jan_sales
@@ -45,74 +47,74 @@ select
  	,sum(nov_net) as nov_net
  	,sum(dec_net) as dec_net
  from (
-    select 
-    w_warehouse_name
-    ,w_warehouse_sq_ft
-    ,w_city
-    ,w_county
-    ,w_state
-    ,w_country
-    ,concat('DIAMOND', ',', 'AIRBORNE') as ship_carriers
-        ,d_year as year
+    (select 
+ 	w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+ 	,'DIAMOND' || ',' || 'AIRBORNE' as ship_carriers
+       ,d_year as year
  	,sum(case when d_moy = 1 
- 		  then ws_sales_price* ws_quantity else 0 end) as jan_sales
- 		  ,sum(case when d_moy = 2 
- 		  	    then ws_sales_price* ws_quantity else 0 end) as feb_sales
- 			    ,sum(case when d_moy = 3 
- 			    	      then ws_sales_price* ws_quantity else 0 end) as mar_sales
- 				      ,sum(case when d_moy = 4 
- 				      		then ws_sales_price* ws_quantity else 0 end) as apr_sales
- 						,sum(case when d_moy = 5 
- 							  then ws_sales_price* ws_quantity else 0 end) as may_sales
- 							  ,sum(case when d_moy = 6 
- 							  	    then ws_sales_price* ws_quantity else 0 end) as jun_sales
- 								    ,sum(case when d_moy = 7 
- 								    	      then ws_sales_price* ws_quantity else 0 end) as jul_sales
- 									      ,sum(case when d_moy = 8 
- 									      		then ws_sales_price* ws_quantity else 0 end) as aug_sales
- 											,sum(case when d_moy = 9 
- 												  then ws_sales_price* ws_quantity else 0 end) as sep_sales
- 												  ,sum(case when d_moy = 10 
- 												  	    then ws_sales_price* ws_quantity else 0 end) as oct_sales
- 													    ,sum(case when d_moy = 11
- 													    	      then ws_sales_price* ws_quantity else 0 end) as nov_sales
- 														      ,sum(case when d_moy = 12
- 														      		then ws_sales_price* ws_quantity else 0 end) as dec_sales
- 																,sum(case when d_moy = 1 
- 																	  then ws_net_paid_inc_tax * ws_quantity else 0 end) as jan_net
- 																	  ,sum(case when d_moy = 2
- 																	  	    then ws_net_paid_inc_tax * ws_quantity else 0 end) as feb_net
- 																		    ,sum(case when d_moy = 3 
- 																		    	      then ws_net_paid_inc_tax * ws_quantity else 0 end) as mar_net
- 																			      ,sum(case when d_moy = 4 
- 																			      		then ws_net_paid_inc_tax * ws_quantity else 0 end) as apr_net
- 																					,sum(case when d_moy = 5 
- 																						  then ws_net_paid_inc_tax * ws_quantity else 0 end) as may_net
- 																						  ,sum(case when d_moy = 6 
- 																						  	    then ws_net_paid_inc_tax * ws_quantity else 0 end) as jun_net
- 																							    ,sum(case when d_moy = 7 
- 																							    	      then ws_net_paid_inc_tax * ws_quantity else 0 end) as jul_net
- 																								      ,sum(case when d_moy = 8 
- 																								      		then ws_net_paid_inc_tax * ws_quantity else 0 end) as aug_net
- 																										,sum(case when d_moy = 9 
- 																											  then ws_net_paid_inc_tax * ws_quantity else 0 end) as sep_net
- 																											  ,sum(case when d_moy = 10 
- 																											  	    then ws_net_paid_inc_tax * ws_quantity else 0 end) as oct_net
- 																												    ,sum(case when d_moy = 11
- 																												    	      then ws_net_paid_inc_tax * ws_quantity else 0 end) as nov_net
- 																													      ,sum(case when d_moy = 12
- 																													      		then ws_net_paid_inc_tax * ws_quantity else 0 end) as dec_net
+ 		then ws_sales_price* ws_quantity else 0 end) as jan_sales
+ 	,sum(case when d_moy = 2 
+ 		then ws_sales_price* ws_quantity else 0 end) as feb_sales
+ 	,sum(case when d_moy = 3 
+ 		then ws_sales_price* ws_quantity else 0 end) as mar_sales
+ 	,sum(case when d_moy = 4 
+ 		then ws_sales_price* ws_quantity else 0 end) as apr_sales
+ 	,sum(case when d_moy = 5 
+ 		then ws_sales_price* ws_quantity else 0 end) as may_sales
+ 	,sum(case when d_moy = 6 
+ 		then ws_sales_price* ws_quantity else 0 end) as jun_sales
+ 	,sum(case when d_moy = 7 
+ 		then ws_sales_price* ws_quantity else 0 end) as jul_sales
+ 	,sum(case when d_moy = 8 
+ 		then ws_sales_price* ws_quantity else 0 end) as aug_sales
+ 	,sum(case when d_moy = 9 
+ 		then ws_sales_price* ws_quantity else 0 end) as sep_sales
+ 	,sum(case when d_moy = 10 
+ 		then ws_sales_price* ws_quantity else 0 end) as oct_sales
+ 	,sum(case when d_moy = 11
+ 		then ws_sales_price* ws_quantity else 0 end) as nov_sales
+ 	,sum(case when d_moy = 12
+ 		then ws_sales_price* ws_quantity else 0 end) as dec_sales
+ 	,sum(case when d_moy = 1 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jan_net
+ 	,sum(case when d_moy = 2
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as feb_net
+ 	,sum(case when d_moy = 3 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as mar_net
+ 	,sum(case when d_moy = 4 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as apr_net
+ 	,sum(case when d_moy = 5 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as may_net
+ 	,sum(case when d_moy = 6 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jun_net
+ 	,sum(case when d_moy = 7 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jul_net
+ 	,sum(case when d_moy = 8 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as aug_net
+ 	,sum(case when d_moy = 9 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as sep_net
+ 	,sum(case when d_moy = 10 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as oct_net
+ 	,sum(case when d_moy = 11
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as nov_net
+ 	,sum(case when d_moy = 12
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as dec_net
      from
           web_sales
          ,warehouse
          ,date_dim
          ,time_dim
- 	   ,ship_mode
+ 	  ,ship_mode
      where
-            web_sales.ws_warehouse_sk =  warehouse.w_warehouse_sk
-        and web_sales.ws_sold_date_sk = date_dim.d_date_sk
-        and web_sales.ws_sold_time_sk = time_dim.t_time_sk
- 	and web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk
+            ws_warehouse_sk =  w_warehouse_sk
+        and ws_sold_date_sk = d_date_sk
+        and ws_sold_time_sk = t_time_sk
+ 	and ws_ship_mode_sk = sm_ship_mode_sk
         and d_year = 2002
  	and t_time between 49530 and 49530+28800 
  	and sm_carrier in ('DIAMOND','AIRBORNE')
@@ -124,75 +126,76 @@ select
  	,w_state
  	,w_country
        ,d_year
+ 	)
  union all
-    select 
-    w_warehouse_name
-    ,w_warehouse_sq_ft
-    ,w_city
-    ,w_county
-    ,w_state
-    ,w_country
-        ,concat('DIAMOND', ',', 'AIRBORNE') as ship_carriers
+    (select 
+ 	w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+ 	,'DIAMOND' || ',' || 'AIRBORNE' as ship_carriers
        ,d_year as year
-       ,sum(case when d_moy = 1 
-       		 then cs_ext_sales_price* cs_quantity else 0 end) as jan_sales
- 		 ,sum(case when d_moy = 2 
- 		 	   then cs_ext_sales_price* cs_quantity else 0 end) as feb_sales
- 			   ,sum(case when d_moy = 3 
- 			   	     then cs_ext_sales_price* cs_quantity else 0 end) as mar_sales
- 				     ,sum(case when d_moy = 4 
- 				     	       then cs_ext_sales_price* cs_quantity else 0 end) as apr_sales
- 					       ,sum(case when d_moy = 5 
- 					       		 then cs_ext_sales_price* cs_quantity else 0 end) as may_sales
- 							 ,sum(case when d_moy = 6 
- 							 	   then cs_ext_sales_price* cs_quantity else 0 end) as jun_sales
- 								   ,sum(case when d_moy = 7 
- 								   	     then cs_ext_sales_price* cs_quantity else 0 end) as jul_sales
- 									     ,sum(case when d_moy = 8 
- 									     	       then cs_ext_sales_price* cs_quantity else 0 end) as aug_sales
- 										       ,sum(case when d_moy = 9 
- 										       		 then cs_ext_sales_price* cs_quantity else 0 end) as sep_sales
- 												 ,sum(case when d_moy = 10 
- 												 	   then cs_ext_sales_price* cs_quantity else 0 end) as oct_sales
- 													   ,sum(case when d_moy = 11
- 													   	     then cs_ext_sales_price* cs_quantity else 0 end) as nov_sales
- 														     ,sum(case when d_moy = 12
- 														     	       then cs_ext_sales_price* cs_quantity else 0 end) as dec_sales
- 															       ,sum(case when d_moy = 1 
- 															       		 then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jan_net
- 																	 ,sum(case when d_moy = 2 
- 																	 	   then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as feb_net
- 																		   ,sum(case when d_moy = 3 
- 																		   	     then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as mar_net
- 																			     ,sum(case when d_moy = 4 
- 																			     	       then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as apr_net
- 																				       ,sum(case when d_moy = 5 
- 																				       		 then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as may_net
- 																						 ,sum(case when d_moy = 6 
- 																						 	   then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jun_net
- 																							   ,sum(case when d_moy = 7 
- 																							   	     then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jul_net
- 																								     ,sum(case when d_moy = 8 
- 																								     	       then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as aug_net
- 																									       ,sum(case when d_moy = 9 
- 																									       		 then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as sep_net
- 																											 ,sum(case when d_moy = 10 
- 																											 	   then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as oct_net
- 																												   ,sum(case when d_moy = 11
- 																												   	     then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as nov_net
- 																													     ,sum(case when d_moy = 12
- 																													     	       then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as dec_net
+ 	,sum(case when d_moy = 1 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jan_sales
+ 	,sum(case when d_moy = 2 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as feb_sales
+ 	,sum(case when d_moy = 3 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as mar_sales
+ 	,sum(case when d_moy = 4 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as apr_sales
+ 	,sum(case when d_moy = 5 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as may_sales
+ 	,sum(case when d_moy = 6 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jun_sales
+ 	,sum(case when d_moy = 7 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jul_sales
+ 	,sum(case when d_moy = 8 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as aug_sales
+ 	,sum(case when d_moy = 9 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as sep_sales
+ 	,sum(case when d_moy = 10 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as oct_sales
+ 	,sum(case when d_moy = 11
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as nov_sales
+ 	,sum(case when d_moy = 12
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as dec_sales
+ 	,sum(case when d_moy = 1 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jan_net
+ 	,sum(case when d_moy = 2 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as feb_net
+ 	,sum(case when d_moy = 3 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as mar_net
+ 	,sum(case when d_moy = 4 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as apr_net
+ 	,sum(case when d_moy = 5 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as may_net
+ 	,sum(case when d_moy = 6 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jun_net
+ 	,sum(case when d_moy = 7 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jul_net
+ 	,sum(case when d_moy = 8 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as aug_net
+ 	,sum(case when d_moy = 9 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as sep_net
+ 	,sum(case when d_moy = 10 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as oct_net
+ 	,sum(case when d_moy = 11
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as nov_net
+ 	,sum(case when d_moy = 12
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as dec_net
      from
           catalog_sales
          ,warehouse
          ,date_dim
          ,time_dim
- 	  ,ship_mode
+ 	 ,ship_mode
      where
-            catalog_sales.cs_warehouse_sk =  warehouse.w_warehouse_sk
-        and catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
-        and catalog_sales.cs_sold_time_sk = time_dim.t_time_sk
- 	and catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk
+            cs_warehouse_sk =  w_warehouse_sk
+        and cs_sold_date_sk = d_date_sk
+        and cs_sold_time_sk = t_time_sk
+ 	and cs_ship_mode_sk = sm_ship_mode_sk
         and d_year = 2002
  	and t_time between 49530 AND 49530+28800 
  	and sm_carrier in ('DIAMOND','AIRBORNE')
@@ -204,6 +207,7 @@ select
  	,w_state
  	,w_country
        ,d_year
+     ) 
  ) x
  group by 
         w_warehouse_name
@@ -217,3 +221,4 @@ select
  order by w_warehouse_name
  limit 100;
 
+-- end query 1 in stream 0 using template query66.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query67.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query67.q b/ql/src/test/queries/clientpositive/perf/query67.q
index ca2fc61..c3ecf2a 100644
--- a/ql/src/test/queries/clientpositive/perf/query67.q
+++ b/ql/src/test/queries/clientpositive/perf/query67.q
@@ -1,5 +1,5 @@
 set hive.mapred.mode=nonstrict;
-
+-- start query 1 in stream 0 using template query67.tpl and seed 1819994127
 explain
 select  *
 from (select i_category
@@ -43,3 +43,4 @@ order by i_category
         ,rk
 limit 100;
 
+-- end query 1 in stream 0 using template query67.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query68.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query68.q b/ql/src/test/queries/clientpositive/perf/query68.q
index 34f87c2..964dc8a 100644
--- a/ql/src/test/queries/clientpositive/perf/query68.q
+++ b/ql/src/test/queries/clientpositive/perf/query68.q
@@ -1 +1,44 @@
-explain select c_last_name ,c_first_name ,ca_city ,bought_city ,ss_ticket_number ,extended_price ,extended_tax ,list_price from (select ss_ticket_number ,ss_customer_sk ,ca_city bought_city ,sum(ss_ext_sales_price) extended_price ,sum(ss_ext_list_price) list_price ,sum(ss_ext_tax) extended_tax from store_sales ,date_dim ,store ,household_demographics ,customer_address where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and store_sales.ss_addr_sk = customer_address.ca_address_sk and date_dim.d_dom between 1 and 2 and (household_demographics.hd_dep_count = 4 or household_demographics.hd_vehicle_count= 2) and date_dim.d_year in (1998,1998+1,1998+2) and store.s_city in ('Rosedale','Bethlehem') group by ss_ticket_number ,ss_customer_sk ,ss_addr_sk,ca_city) dn ,customer ,customer_address current_addr where dn.ss_customer_sk = customer.c_customer_sk and customer.c_current_addr_
 sk = current_addr.ca_address_sk and current_addr.ca_city <> bought_city order by c_last_name ,ss_ticket_number limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query68.tpl and seed 803547492
+explain
+select  c_last_name
+       ,c_first_name
+       ,ca_city
+       ,bought_city
+       ,ss_ticket_number
+       ,extended_price
+       ,extended_tax
+       ,list_price
+ from (select ss_ticket_number
+             ,ss_customer_sk
+             ,ca_city bought_city
+             ,sum(ss_ext_sales_price) extended_price 
+             ,sum(ss_ext_list_price) list_price
+             ,sum(ss_ext_tax) extended_tax 
+       from store_sales
+           ,date_dim
+           ,store
+           ,household_demographics
+           ,customer_address 
+       where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+         and store_sales.ss_store_sk = store.s_store_sk  
+        and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+        and store_sales.ss_addr_sk = customer_address.ca_address_sk
+        and date_dim.d_dom between 1 and 2 
+        and (household_demographics.hd_dep_count = 2 or
+             household_demographics.hd_vehicle_count= 1)
+        and date_dim.d_year in (1998,1998+1,1998+2)
+        and store.s_city in ('Cedar Grove','Wildwood')
+       group by ss_ticket_number
+               ,ss_customer_sk
+               ,ss_addr_sk,ca_city) dn
+      ,customer
+      ,customer_address current_addr
+ where ss_customer_sk = c_customer_sk
+   and customer.c_current_addr_sk = current_addr.ca_address_sk
+   and current_addr.ca_city <> bought_city
+ order by c_last_name
+         ,ss_ticket_number
+ limit 100;
+
+-- end query 1 in stream 0 using template query68.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query69.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query69.q b/ql/src/test/queries/clientpositive/perf/query69.q
index d9528ae..ce2d19c 100644
--- a/ql/src/test/queries/clientpositive/perf/query69.q
+++ b/ql/src/test/queries/clientpositive/perf/query69.q
@@ -1,6 +1,7 @@
 set hive.mapred.mode=nonstrict;
-
-explain select  
+-- start query 1 in stream 0 using template query69.tpl and seed 797269820
+explain
+select  
   cd_gender,
   cd_marital_status,
   cd_education_status,
@@ -44,3 +45,5 @@ explain select
           cd_purchase_estimate,
           cd_credit_rating
  limit 100;
+
+-- end query 1 in stream 0 using template query69.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query7.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query7.q b/ql/src/test/queries/clientpositive/perf/query7.q
index 39c4383..7bc1a00 100644
--- a/ql/src/test/queries/clientpositive/perf/query7.q
+++ b/ql/src/test/queries/clientpositive/perf/query7.q
@@ -1 +1,23 @@
-explain select i_item_id, avg(ss_quantity) agg1, avg(ss_list_price) agg2, avg(ss_coupon_amt) agg3, avg(ss_sales_price) agg4 from store_sales, customer_demographics, date_dim, item, promotion where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_item_sk = item.i_item_sk and store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk and store_sales.ss_promo_sk = promotion.p_promo_sk and cd_gender = 'F' and cd_marital_status = 'W' and cd_education_status = 'Primary' and (p_channel_email = 'N' or p_channel_event = 'N') and d_year = 1998 group by i_item_id order by i_item_id limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query7.tpl and seed 1930872976
+explain
+select  i_item_id, 
+        avg(ss_quantity) agg1,
+        avg(ss_list_price) agg2,
+        avg(ss_coupon_amt) agg3,
+        avg(ss_sales_price) agg4 
+ from store_sales, customer_demographics, date_dim, item, promotion
+ where ss_sold_date_sk = d_date_sk and
+       ss_item_sk = i_item_sk and
+       ss_cdemo_sk = cd_demo_sk and
+       ss_promo_sk = p_promo_sk and
+       cd_gender = 'F' and 
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       (p_channel_email = 'N' or p_channel_event = 'N') and
+       d_year = 1998 
+ group by i_item_id
+ order by i_item_id
+ limit 100;
+
+-- end query 1 in stream 0 using template query7.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query70.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query70.q b/ql/src/test/queries/clientpositive/perf/query70.q
index 80f8c23..7974976 100644
--- a/ql/src/test/queries/clientpositive/perf/query70.q
+++ b/ql/src/test/queries/clientpositive/perf/query70.q
@@ -1,5 +1,5 @@
 set hive.mapred.mode=nonstrict;
-
+-- start query 1 in stream 0 using template query70.tpl and seed 1819994127
 explain
 select  
     sum(ss_net_profit) as total_sum
@@ -7,25 +7,25 @@ select
    ,s_county
    ,grouping(s_state)+grouping(s_county) as lochierarchy
    ,rank() over (
-     partition by grouping(s_state)+grouping(s_county),
-     case when grouping(s_county) = 0 then s_state end 
-     order by sum(ss_net_profit) desc) as rank_within_parent
+ 	partition by grouping(s_state)+grouping(s_county),
+ 	case when grouping(s_county) = 0 then s_state end 
+ 	order by sum(ss_net_profit) desc) as rank_within_parent
  from
     store_sales
    ,date_dim       d1
-   ,store s
+   ,store
  where
     d1.d_month_seq between 1212 and 1212+11
  and d1.d_date_sk = ss_sold_date_sk
  and s_store_sk  = ss_store_sk
- and s.s_state in
+ and s_state in
              ( select s_state
                from  (select s_state as s_state,
-                 rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking
+ 			    rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking
                       from   store_sales, store, date_dim
                       where  d_month_seq between 1212 and 1212+11
-                 and d_date_sk = ss_sold_date_sk
-                 and s_store_sk  = ss_store_sk
+ 			    and d_date_sk = ss_sold_date_sk
+ 			    and s_store_sk  = ss_store_sk
                       group by s_state
                      ) tmp1 
                where ranking <= 5
@@ -37,3 +37,4 @@ select
   ,rank_within_parent
  limit 100;
 
+-- end query 1 in stream 0 using template query70.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query71.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query71.q b/ql/src/test/queries/clientpositive/perf/query71.q
index d0601d2..ea6548e 100644
--- a/ql/src/test/queries/clientpositive/perf/query71.q
+++ b/ql/src/test/queries/clientpositive/perf/query71.q
@@ -1,2 +1,42 @@
 set hive.mapred.mode=nonstrict;
-explain select i_brand_id brand_id, i_brand brand,t_hour,t_minute, sum(ext_price) ext_price from item JOIN (select ws_ext_sales_price as ext_price, ws_sold_date_sk as sold_date_sk, ws_item_sk as sold_item_sk, ws_sold_time_sk as time_sk from web_sales,date_dim where date_dim.d_date_sk = web_sales.ws_sold_date_sk and d_moy=12 and d_year=2001 union all select cs_ext_sales_price as ext_price, cs_sold_date_sk as sold_date_sk, cs_item_sk as sold_item_sk, cs_sold_time_sk as time_sk from catalog_sales,date_dim where date_dim.d_date_sk = catalog_sales.cs_sold_date_sk and d_moy=12 and d_year=2001 union all select ss_ext_sales_price as ext_price, ss_sold_date_sk as sold_date_sk, ss_item_sk as sold_item_sk, ss_sold_time_sk as time_sk from store_sales,date_dim where date_dim.d_date_sk = store_sales.ss_sold_date_sk and d_moy=12 and d_year=2001 ) tmp ON tmp.sold_item_sk = item.i_item_sk JOIN time_dim ON tmp.time_sk = time_dim.t_time_sk where i_manager_id=1 and (t_meal_time = 'breakfast' or t_meal_
 time = 'dinner') group by i_brand, i_brand_id,t_hour,t_minute order by ext_price desc, i_brand_id ;
+-- start query 1 in stream 0 using template query71.tpl and seed 2031708268
+explain
+select i_brand_id brand_id, i_brand brand,t_hour,t_minute,
+ 	sum(ext_price) ext_price
+ from item, (select ws_ext_sales_price as ext_price, 
+                        ws_sold_date_sk as sold_date_sk,
+                        ws_item_sk as sold_item_sk,
+                        ws_sold_time_sk as time_sk  
+                 from web_sales,date_dim
+                 where d_date_sk = ws_sold_date_sk
+                   and d_moy=12
+                   and d_year=2001
+                 union all
+                 select cs_ext_sales_price as ext_price,
+                        cs_sold_date_sk as sold_date_sk,
+                        cs_item_sk as sold_item_sk,
+                        cs_sold_time_sk as time_sk
+                 from catalog_sales,date_dim
+                 where d_date_sk = cs_sold_date_sk
+                   and d_moy=12
+                   and d_year=2001
+                 union all
+                 select ss_ext_sales_price as ext_price,
+                        ss_sold_date_sk as sold_date_sk,
+                        ss_item_sk as sold_item_sk,
+                        ss_sold_time_sk as time_sk
+                 from store_sales,date_dim
+                 where d_date_sk = ss_sold_date_sk
+                   and d_moy=12
+                   and d_year=2001
+                 ) as tmp,time_dim
+ where
+   sold_item_sk = i_item_sk
+   and i_manager_id=1
+   and time_sk = t_time_sk
+   and (t_meal_time = 'breakfast' or t_meal_time = 'dinner')
+ group by i_brand, i_brand_id,t_hour,t_minute
+ order by ext_price desc, i_brand_id
+ ;
+
+-- end query 1 in stream 0 using template query71.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query72.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query72.q b/ql/src/test/queries/clientpositive/perf/query72.q
index 6c076d6..20fbcb1 100644
--- a/ql/src/test/queries/clientpositive/perf/query72.q
+++ b/ql/src/test/queries/clientpositive/perf/query72.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query72.tpl and seed 2031708268
 explain
 select  i_item_desc
       ,w_warehouse_name
@@ -28,3 +30,4 @@ group by i_item_desc,w_warehouse_name,d1.d_week_seq
 order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq
 limit 100;
 
+-- end query 1 in stream 0 using template query72.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query73.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query73.q b/ql/src/test/queries/clientpositive/perf/query73.q
index 95c4695..42ccaa1 100644
--- a/ql/src/test/queries/clientpositive/perf/query73.q
+++ b/ql/src/test/queries/clientpositive/perf/query73.q
@@ -1,2 +1,30 @@
 set hive.mapred.mode=nonstrict;
-explain select c_last_name ,c_first_name ,c_salutation ,c_preferred_cust_flag ,ss_ticket_number ,cnt from (select ss_ticket_number ,ss_customer_sk ,count(*) cnt from store_sales,date_dim,store,household_demographics where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and date_dim.d_dom between 1 and 2 and (household_demographics.hd_buy_potential = '1001-5000' or household_demographics.hd_buy_potential = '5001-10000') and household_demographics.hd_vehicle_count > 0 and case when household_demographics.hd_vehicle_count > 0 then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 and date_dim.d_year in (1998,1998+1,1998+2) and store.s_county in ('Kittitas County','Adams County','Richland County','Furnas County') group by ss_ticket_number,ss_customer_sk) dj,customer where dj.ss_customer_sk = customer.c_customer_sk and cnt between 1 
 and 5 order by cnt desc;
+-- start query 1 in stream 0 using template query73.tpl and seed 1971067816
+explain
+select c_last_name
+       ,c_first_name
+       ,c_salutation
+       ,c_preferred_cust_flag 
+       ,ss_ticket_number
+       ,cnt from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,count(*) cnt
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and date_dim.d_dom between 1 and 2 
+    and (household_demographics.hd_buy_potential = '>10000' or
+         household_demographics.hd_buy_potential = 'unknown')
+    and household_demographics.hd_vehicle_count > 0
+    and case when household_demographics.hd_vehicle_count > 0 then 
+             household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1
+    and date_dim.d_year in (2000,2000+1,2000+2)
+    and store.s_county in ('Mobile County','Maverick County','Huron County','Kittitas County')
+    group by ss_ticket_number,ss_customer_sk) dj,customer
+    where ss_customer_sk = c_customer_sk
+      and cnt between 1 and 5
+    order by cnt desc;
+
+-- end query 1 in stream 0 using template query73.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query75.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query75.q b/ql/src/test/queries/clientpositive/perf/query75.q
index a815c9e..ac1fc38 100644
--- a/ql/src/test/queries/clientpositive/perf/query75.q
+++ b/ql/src/test/queries/clientpositive/perf/query75.q
@@ -1 +1,72 @@
-explain WITH all_sales AS ( SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,SUM(sales_cnt) AS sales_cnt ,SUM(sales_amt) AS sales_amt FROM (SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk JOIN date_dim ON d_date_sk=cs_sold_date_sk LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number AND cs_item_sk=cr_item_sk) WHERE i_category='Sports' UNION ALL SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt FROM store_sales JOIN item ON i_item_sk=ss_item_sk JOIN date_dim ON d_date_sk=ss_sold_date_sk LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number AND ss_item_sk=sr_item_sk) WHERE i_category='Sports' UNION ALL SELECT d_year 
 ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt FROM web_sales JOIN item ON i_item_sk=ws_item_sk JOIN date_dim ON d_date_sk=ws_sold_date_sk LEFT JOIN web_returns ON (ws_order_number=wr_order_number AND ws_item_sk=wr_item_sk) WHERE i_category='Sports') sales_detail GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) SELECT prev_yr.d_year AS prev_year ,curr_yr.d_year AS year ,curr_yr.i_brand_id ,curr_yr.i_class_id ,curr_yr.i_category_id ,curr_yr.i_manufact_id ,prev_yr.sales_cnt AS prev_yr_cnt ,curr_yr.sales_cnt AS curr_yr_cnt ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff FROM all_sales curr_yr, all_sales prev_yr WHERE curr_yr.i_brand_id=prev_yr.i_brand_id AND curr_yr.i_class_id=prev_yr.i_class_id AND curr_yr.i_category_id=prev_yr.i_category_id AND curr_yr.i_manufact_id=prev_yr.
 i_manufact_id AND curr_yr.d_year=2002 AND prev_yr.d_year=2002-1 AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9 ORDER BY sales_cnt_diff limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query75.tpl and seed 1819994127
+explain
+WITH all_sales AS (
+ SELECT d_year
+       ,i_brand_id
+       ,i_class_id
+       ,i_category_id
+       ,i_manufact_id
+       ,SUM(sales_cnt) AS sales_cnt
+       ,SUM(sales_amt) AS sales_amt
+ FROM (SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt
+             ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt
+       FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk
+                          JOIN date_dim ON d_date_sk=cs_sold_date_sk
+                          LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number 
+                                                    AND cs_item_sk=cr_item_sk)
+       WHERE i_category='Sports'
+       UNION
+       SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt
+             ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt
+       FROM store_sales JOIN item ON i_item_sk=ss_item_sk
+                        JOIN date_dim ON d_date_sk=ss_sold_date_sk
+                        LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number 
+                                                AND ss_item_sk=sr_item_sk)
+       WHERE i_category='Sports'
+       UNION
+       SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt
+             ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt
+       FROM web_sales JOIN item ON i_item_sk=ws_item_sk
+                      JOIN date_dim ON d_date_sk=ws_sold_date_sk
+                      LEFT JOIN web_returns ON (ws_order_number=wr_order_number 
+                                            AND ws_item_sk=wr_item_sk)
+       WHERE i_category='Sports') sales_detail
+ GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id)
+ SELECT  prev_yr.d_year AS prev_year
+                          ,curr_yr.d_year AS year
+                          ,curr_yr.i_brand_id
+                          ,curr_yr.i_class_id
+                          ,curr_yr.i_category_id
+                          ,curr_yr.i_manufact_id
+                          ,prev_yr.sales_cnt AS prev_yr_cnt
+                          ,curr_yr.sales_cnt AS curr_yr_cnt
+                          ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff
+                          ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff
+ FROM all_sales curr_yr, all_sales prev_yr
+ WHERE curr_yr.i_brand_id=prev_yr.i_brand_id
+   AND curr_yr.i_class_id=prev_yr.i_class_id
+   AND curr_yr.i_category_id=prev_yr.i_category_id
+   AND curr_yr.i_manufact_id=prev_yr.i_manufact_id
+   AND curr_yr.d_year=2002
+   AND prev_yr.d_year=2002-1
+   AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9
+ ORDER BY sales_cnt_diff
+ limit 100;
+
+-- end query 1 in stream 0 using template query75.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query76.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query76.q b/ql/src/test/queries/clientpositive/perf/query76.q
index 1894cf4..ca943ce 100644
--- a/ql/src/test/queries/clientpositive/perf/query76.q
+++ b/ql/src/test/queries/clientpositive/perf/query76.q
@@ -1 +1,26 @@
-explain select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( SELECT 'store' as channel, 'ss_addr_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price FROM store_sales, item, date_dim WHERE ss_addr_sk IS NULL AND store_sales.ss_sold_date_sk=date_dim.d_date_sk AND store_sales.ss_item_sk=item.i_item_sk UNION ALL SELECT 'web' as channel, 'ws_web_page_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price FROM web_sales, item, date_dim WHERE ws_web_page_sk IS NULL AND web_sales.ws_sold_date_sk=date_dim.d_date_sk AND web_sales.ws_item_sk=item.i_item_sk UNION ALL SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price FROM catalog_sales, item, date_dim WHERE cs_warehouse_sk IS NULL AND catalog_sales.cs_sold_date_sk=date_dim.d_date_sk AND catalog_sales.cs_item_sk=item.i_item_sk) foo GROUP BY channel, col_name, d_year, d_qoy, i_c
 ategory ORDER BY channel, col_name, d_year, d_qoy, i_category limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query76.tpl and seed 2031708268
+explain
+select  channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM (
+        SELECT 'store' as channel, 'ss_addr_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price
+         FROM store_sales, item, date_dim
+         WHERE ss_addr_sk IS NULL
+           AND ss_sold_date_sk=d_date_sk
+           AND ss_item_sk=i_item_sk
+        UNION ALL
+        SELECT 'web' as channel, 'ws_web_page_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price
+         FROM web_sales, item, date_dim
+         WHERE ws_web_page_sk IS NULL
+           AND ws_sold_date_sk=d_date_sk
+           AND ws_item_sk=i_item_sk
+        UNION ALL
+        SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price
+         FROM catalog_sales, item, date_dim
+         WHERE cs_warehouse_sk IS NULL
+           AND cs_sold_date_sk=d_date_sk
+           AND cs_item_sk=i_item_sk) foo
+GROUP BY channel, col_name, d_year, d_qoy, i_category
+ORDER BY channel, col_name, d_year, d_qoy, i_category
+limit 100;
+
+-- end query 1 in stream 0 using template query76.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query77.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query77.q b/ql/src/test/queries/clientpositive/perf/query77.q
new file mode 100644
index 0000000..2857813
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query77.q
@@ -0,0 +1,109 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query77.tpl and seed 1819994127
+explain
+with ss as
+ (select s_store_sk,
+         sum(ss_ext_sales_price) as sales,
+         sum(ss_net_profit) as profit
+ from store_sales,
+      date_dim,
+      store
+ where ss_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date) 
+                  and (cast('1998-08-04' as date) +  30 days) 
+       and ss_store_sk = s_store_sk
+ group by s_store_sk)
+ ,
+ sr as
+ (select s_store_sk,
+         sum(sr_return_amt) as returns,
+         sum(sr_net_loss) as profit_loss
+ from store_returns,
+      date_dim,
+      store
+ where sr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and sr_store_sk = s_store_sk
+ group by s_store_sk), 
+ cs as
+ (select cs_call_center_sk,
+        sum(cs_ext_sales_price) as sales,
+        sum(cs_net_profit) as profit
+ from catalog_sales,
+      date_dim
+ where cs_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+ group by cs_call_center_sk 
+ ), 
+ cr as
+ (select
+        sum(cr_return_amount) as returns,
+        sum(cr_net_loss) as profit_loss
+ from catalog_returns,
+      date_dim
+ where cr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+ ), 
+ ws as
+ ( select wp_web_page_sk,
+        sum(ws_ext_sales_price) as sales,
+        sum(ws_net_profit) as profit
+ from web_sales,
+      date_dim,
+      web_page
+ where ws_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and ws_web_page_sk = wp_web_page_sk
+ group by wp_web_page_sk), 
+ wr as
+ (select wp_web_page_sk,
+        sum(wr_return_amt) as returns,
+        sum(wr_net_loss) as profit_loss
+ from web_returns,
+      date_dim,
+      web_page
+ where wr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and wr_web_page_sk = wp_web_page_sk
+ group by wp_web_page_sk)
+  select  channel
+        , id
+        , sum(sales) as sales
+        , sum(returns) as returns
+        , sum(profit) as profit
+ from 
+ (select 'store channel' as channel
+        , ss.s_store_sk as id
+        , sales
+        , coalesce(returns, 0) as returns
+        , (profit - coalesce(profit_loss,0)) as profit
+ from   ss left join sr
+        on  ss.s_store_sk = sr.s_store_sk
+ union all
+ select 'catalog channel' as channel
+        , cs_call_center_sk as id
+        , sales
+        , returns
+        , (profit - profit_loss) as profit
+ from  cs
+       , cr
+ union all
+ select 'web channel' as channel
+        , ws.wp_web_page_sk as id
+        , sales
+        , coalesce(returns, 0) returns
+        , (profit - coalesce(profit_loss,0)) as profit
+ from   ws left join wr
+        on  ws.wp_web_page_sk = wr.wp_web_page_sk
+ ) x
+ group by rollup (channel, id)
+ order by channel
+         ,id
+ limit 100;
+
+-- end query 1 in stream 0 using template query77.tpl


[10/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query46.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query46.q.out b/ql/src/test/results/clientpositive/perf/query46.q.out
index 8c6e914..6806703 100644
--- a/ql/src/test/results/clientpositive/perf/query46.q.out
+++ b/ql/src/test/results/clientpositive/perf/query46.q.out
@@ -1,6 +1,70 @@
-PREHOOK: query: explain select c_last_name ,c_first_name ,ca_city ,bought_city ,ss_ticket_number ,amt,profit from (select ss_ticket_number ,ss_customer_sk ,ca_city bought_city ,sum(ss_coupon_amt) amt ,sum(ss_net_profit) profit from store_sales,date_dim,store,household_demographics,customer_address where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and store_sales.ss_addr_sk = customer_address.ca_address_sk and (household_demographics.hd_dep_count = 4 or household_demographics.hd_vehicle_count= 2) and date_dim.d_dow in (6,0) and date_dim.d_year in (1998,1998+1,1998+2) and store.s_city in ('Rosedale','Bethlehem','Clinton','Clifton','Springfield') group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr where dn.ss_customer_sk = customer.c_customer_sk and customer.c_current_addr_sk = current_addr.ca_address_sk and current_addr.
 ca_city <> bought_city order by c_last_name ,c_first_name ,ca_city ,bought_city ,ss_ticket_number limit 100
+PREHOOK: query: explain
+select  c_last_name
+       ,c_first_name
+       ,ca_city
+       ,bought_city
+       ,ss_ticket_number
+       ,amt,profit 
+ from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,ca_city bought_city
+          ,sum(ss_coupon_amt) amt
+          ,sum(ss_net_profit) profit
+    from store_sales,date_dim,store,household_demographics,customer_address 
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and store_sales.ss_addr_sk = customer_address.ca_address_sk
+    and (household_demographics.hd_dep_count = 2 or
+         household_demographics.hd_vehicle_count= 1)
+    and date_dim.d_dow in (6,0)
+    and date_dim.d_year in (1998,1998+1,1998+2) 
+    and store.s_city in ('Cedar Grove','Wildwood','Union','Salem','Highland Park') 
+    group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr
+    where ss_customer_sk = c_customer_sk
+      and customer.c_current_addr_sk = current_addr.ca_address_sk
+      and current_addr.ca_city <> bought_city
+  order by c_last_name
+          ,c_first_name
+          ,ca_city
+          ,bought_city
+          ,ss_ticket_number
+  limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select c_last_name ,c_first_name ,ca_city ,bought_city ,ss_ticket_number ,amt,profit from (select ss_ticket_number ,ss_customer_sk ,ca_city bought_city ,sum(ss_coupon_amt) amt ,sum(ss_net_profit) profit from store_sales,date_dim,store,household_demographics,customer_address where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and store_sales.ss_addr_sk = customer_address.ca_address_sk and (household_demographics.hd_dep_count = 4 or household_demographics.hd_vehicle_count= 2) and date_dim.d_dow in (6,0) and date_dim.d_year in (1998,1998+1,1998+2) and store.s_city in ('Rosedale','Bethlehem','Clinton','Clifton','Springfield') group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr where dn.ss_customer_sk = customer.c_customer_sk and customer.c_current_addr_sk = current_addr.ca_address_sk and current_addr
 .ca_city <> bought_city order by c_last_name ,c_first_name ,ca_city ,bought_city ,ss_ticket_number limit 100
+POSTHOOK: query: explain
+select  c_last_name
+       ,c_first_name
+       ,ca_city
+       ,bought_city
+       ,ss_ticket_number
+       ,amt,profit 
+ from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,ca_city bought_city
+          ,sum(ss_coupon_amt) amt
+          ,sum(ss_net_profit) profit
+    from store_sales,date_dim,store,household_demographics,customer_address 
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and store_sales.ss_addr_sk = customer_address.ca_address_sk
+    and (household_demographics.hd_dep_count = 2 or
+         household_demographics.hd_vehicle_count= 1)
+    and date_dim.d_dow in (6,0)
+    and date_dim.d_year in (1998,1998+1,1998+2) 
+    and store.s_city in ('Cedar Grove','Wildwood','Union','Salem','Highland Park') 
+    group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr
+    where ss_customer_sk = c_customer_sk
+      and customer.c_current_addr_sk = current_addr.ca_address_sk
+      and current_addr.ca_city <> bought_city
+  order by c_last_name
+          ,c_first_name
+          ,ca_city
+          ,bought_city
+          ,ss_ticket_number
+  limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -88,7 +152,7 @@ Stage-0
                                       Select Operator [SEL_17] (rows=7200 width=107)
                                         Output:["_col0"]
                                         Filter Operator [FIL_79] (rows=7200 width=107)
-                                          predicate:(((hd_dep_count = 4) or (hd_vehicle_count = 2)) and hd_demo_sk is not null)
+                                          predicate:(((hd_dep_count = 2) or (hd_vehicle_count = 1)) and hd_demo_sk is not null)
                                           TableScan [TS_15] (rows=7200 width=107)
                                             default@household_demographics,household_demographics,Tbl:COMPLETE,Col:NONE,Output:["hd_demo_sk","hd_dep_count","hd_vehicle_count"]
                                   <-Reducer 10 [SIMPLE_EDGE]
@@ -102,7 +166,7 @@ Stage-0
                                           Select Operator [SEL_14] (rows=852 width=1910)
                                             Output:["_col0"]
                                             Filter Operator [FIL_78] (rows=852 width=1910)
-                                              predicate:((s_city) IN ('Rosedale', 'Bethlehem', 'Clinton', 'Clifton', 'Springfield') and s_store_sk is not null)
+                                              predicate:((s_city) IN ('Cedar Grove', 'Wildwood', 'Union', 'Salem', 'Highland Park') and s_store_sk is not null)
                                               TableScan [TS_12] (rows=1704 width=1910)
                                                 default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_city"]
                                       <-Reducer 9 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query48.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query48.q.out b/ql/src/test/results/clientpositive/perf/query48.q.out
index e377e3f..ffe80b4 100644
--- a/ql/src/test/results/clientpositive/perf/query48.q.out
+++ b/ql/src/test/results/clientpositive/perf/query48.q.out
@@ -1,6 +1,132 @@
-PREHOOK: query: explain select sum (ss_quantity) from store_sales, store, customer_demographics, customer_address, date_dim where store.s_store_sk = store_sales.ss_store_sk and store_sales.ss_sold_date_sk = date_dim.d_date_sk and d_year = 1998 and ( ( customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and ss_sales_price between 100.00 and 150.00 ) or ( customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and ss_sales_price between 50.00 and 100.00 ) or ( customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and ss_sales_price between 150.00 and 200.00 ) ) and ( ( store_sales.ss_addr_sk = customer_address.ca_address_sk and ca_country = 'United States' and ca_state in ('KY', 'GA', 'NM') and ss_net_profit between 0 and 2000 ) or (store_sales.ss_addr_sk = customer_add
 ress.ca_address_sk and ca_country = 'United States' and ca_state in ('MT', 'OR', 'IN') and ss_net_profit between 150 and 3000 ) or (store_sales.ss_addr_sk = customer_address.ca_address_sk and ca_country = 'United States' and ca_state in ('WI', 'MO', 'WV') and ss_net_profit between 50 and 25000 ) )
+PREHOOK: query: explain
+select sum (ss_quantity)
+ from store_sales, store, customer_demographics, customer_address, date_dim
+ where s_store_sk = ss_store_sk
+ and  ss_sold_date_sk = d_date_sk and d_year = 1998
+ and  
+ (
+  (
+   cd_demo_sk = ss_cdemo_sk
+   and 
+   cd_marital_status = 'M'
+   and 
+   cd_education_status = '4 yr Degree'
+   and 
+   ss_sales_price between 100.00 and 150.00  
+   )
+ or
+  (
+  cd_demo_sk = ss_cdemo_sk
+   and 
+   cd_marital_status = 'M'
+   and 
+   cd_education_status = '4 yr Degree'
+   and 
+   ss_sales_price between 50.00 and 100.00   
+  )
+ or 
+ (
+  cd_demo_sk = ss_cdemo_sk
+  and 
+   cd_marital_status = 'M'
+   and 
+   cd_education_status = '4 yr Degree'
+   and 
+   ss_sales_price between 150.00 and 200.00  
+ )
+ )
+ and
+ (
+  (
+  ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('KY', 'GA', 'NM')
+  and ss_net_profit between 0 and 2000  
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('MT', 'OR', 'IN')
+  and ss_net_profit between 150 and 3000 
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('WI', 'MO', 'WV')
+  and ss_net_profit between 50 and 25000 
+  )
+ )
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select sum (ss_quantity) from store_sales, store, customer_demographics, customer_address, date_dim where store.s_store_sk = store_sales.ss_store_sk and store_sales.ss_sold_date_sk = date_dim.d_date_sk and d_year = 1998 and ( ( customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and ss_sales_price between 100.00 and 150.00 ) or ( customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and ss_sales_price between 50.00 and 100.00 ) or ( customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and ss_sales_price between 150.00 and 200.00 ) ) and ( ( store_sales.ss_addr_sk = customer_address.ca_address_sk and ca_country = 'United States' and ca_state in ('KY', 'GA', 'NM') and ss_net_profit between 0 and 2000 ) or (store_sales.ss_addr_sk = customer_ad
 dress.ca_address_sk and ca_country = 'United States' and ca_state in ('MT', 'OR', 'IN') and ss_net_profit between 150 and 3000 ) or (store_sales.ss_addr_sk = customer_address.ca_address_sk and ca_country = 'United States' and ca_state in ('WI', 'MO', 'WV') and ss_net_profit between 50 and 25000 ) )
+POSTHOOK: query: explain
+select sum (ss_quantity)
+ from store_sales, store, customer_demographics, customer_address, date_dim
+ where s_store_sk = ss_store_sk
+ and  ss_sold_date_sk = d_date_sk and d_year = 1998
+ and  
+ (
+  (
+   cd_demo_sk = ss_cdemo_sk
+   and 
+   cd_marital_status = 'M'
+   and 
+   cd_education_status = '4 yr Degree'
+   and 
+   ss_sales_price between 100.00 and 150.00  
+   )
+ or
+  (
+  cd_demo_sk = ss_cdemo_sk
+   and 
+   cd_marital_status = 'M'
+   and 
+   cd_education_status = '4 yr Degree'
+   and 
+   ss_sales_price between 50.00 and 100.00   
+  )
+ or 
+ (
+  cd_demo_sk = ss_cdemo_sk
+  and 
+   cd_marital_status = 'M'
+   and 
+   cd_education_status = '4 yr Degree'
+   and 
+   ss_sales_price between 150.00 and 200.00  
+ )
+ )
+ and
+ (
+  (
+  ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('KY', 'GA', 'NM')
+  and ss_net_profit between 0 and 2000  
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('MT', 'OR', 'IN')
+  and ss_net_profit between 150 and 3000 
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('WI', 'MO', 'WV')
+  and ss_net_profit between 50 and 25000 
+  )
+ )
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query49.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query49.q.out b/ql/src/test/results/clientpositive/perf/query49.q.out
new file mode 100644
index 0000000..8b8ad8b
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query49.q.out
@@ -0,0 +1,504 @@
+PREHOOK: query: explain
+select  
+ 'web' as channel
+ ,web.item
+ ,web.return_ratio
+ ,web.return_rank
+ ,web.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select ws.ws_item_sk as item
+ 		,(cast(sum(coalesce(wr.wr_return_quantity,0)) as dec(15,4))/
+ 		cast(sum(coalesce(ws.ws_quantity,0)) as dec(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(wr.wr_return_amt,0)) as dec(15,4))/
+ 		cast(sum(coalesce(ws.ws_net_paid,0)) as dec(15,4) )) as currency_ratio
+ 		from 
+ 		 web_sales ws left outer join web_returns wr 
+ 			on (ws.ws_order_number = wr.wr_order_number and 
+ 			ws.ws_item_sk = wr.wr_item_sk)
+                 ,date_dim
+ 		where 
+ 			wr.wr_return_amt > 10000 
+ 			and ws.ws_net_profit > 1
+                         and ws.ws_net_paid > 0
+                         and ws.ws_quantity > 0
+                         and ws_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+ 		group by ws.ws_item_sk
+ 	) in_web
+ ) web
+ where 
+ (
+ web.return_rank <= 10
+ or
+ web.currency_rank <= 10
+ )
+ union
+ select 
+ 'catalog' as channel
+ ,catalog.item
+ ,catalog.return_ratio
+ ,catalog.return_rank
+ ,catalog.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select 
+ 		cs.cs_item_sk as item
+ 		,(cast(sum(coalesce(cr.cr_return_quantity,0)) as dec(15,4))/
+ 		cast(sum(coalesce(cs.cs_quantity,0)) as dec(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(cr.cr_return_amount,0)) as dec(15,4))/
+ 		cast(sum(coalesce(cs.cs_net_paid,0)) as dec(15,4) )) as currency_ratio
+ 		from 
+ 		catalog_sales cs left outer join catalog_returns cr
+ 			on (cs.cs_order_number = cr.cr_order_number and 
+ 			cs.cs_item_sk = cr.cr_item_sk)
+                ,date_dim
+ 		where 
+ 			cr.cr_return_amount > 10000 
+ 			and cs.cs_net_profit > 1
+                         and cs.cs_net_paid > 0
+                         and cs.cs_quantity > 0
+                         and cs_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+                 group by cs.cs_item_sk
+ 	) in_cat
+ ) catalog
+ where 
+ (
+ catalog.return_rank <= 10
+ or
+ catalog.currency_rank <=10
+ )
+ union
+ select 
+ 'store' as channel
+ ,store.item
+ ,store.return_ratio
+ ,store.return_rank
+ ,store.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select sts.ss_item_sk as item
+ 		,(cast(sum(coalesce(sr.sr_return_quantity,0)) as dec(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as dec(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(sr.sr_return_amt,0)) as dec(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as dec(15,4) )) as currency_ratio
+ 		from 
+ 		store_sales sts left outer join store_returns sr
+ 			on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk)
+                ,date_dim
+ 		where 
+ 			sr.sr_return_amt > 10000 
+ 			and sts.ss_net_profit > 1
+                         and sts.ss_net_paid > 0 
+                         and sts.ss_quantity > 0
+                         and ss_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+ 		group by sts.ss_item_sk
+ 	) in_store
+ ) store
+ where  (
+ store.return_rank <= 10
+ or 
+ store.currency_rank <= 10
+ )
+ order by 1,4,5
+ limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select  
+ 'web' as channel
+ ,web.item
+ ,web.return_ratio
+ ,web.return_rank
+ ,web.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select ws.ws_item_sk as item
+ 		,(cast(sum(coalesce(wr.wr_return_quantity,0)) as dec(15,4))/
+ 		cast(sum(coalesce(ws.ws_quantity,0)) as dec(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(wr.wr_return_amt,0)) as dec(15,4))/
+ 		cast(sum(coalesce(ws.ws_net_paid,0)) as dec(15,4) )) as currency_ratio
+ 		from 
+ 		 web_sales ws left outer join web_returns wr 
+ 			on (ws.ws_order_number = wr.wr_order_number and 
+ 			ws.ws_item_sk = wr.wr_item_sk)
+                 ,date_dim
+ 		where 
+ 			wr.wr_return_amt > 10000 
+ 			and ws.ws_net_profit > 1
+                         and ws.ws_net_paid > 0
+                         and ws.ws_quantity > 0
+                         and ws_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+ 		group by ws.ws_item_sk
+ 	) in_web
+ ) web
+ where 
+ (
+ web.return_rank <= 10
+ or
+ web.currency_rank <= 10
+ )
+ union
+ select 
+ 'catalog' as channel
+ ,catalog.item
+ ,catalog.return_ratio
+ ,catalog.return_rank
+ ,catalog.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select 
+ 		cs.cs_item_sk as item
+ 		,(cast(sum(coalesce(cr.cr_return_quantity,0)) as dec(15,4))/
+ 		cast(sum(coalesce(cs.cs_quantity,0)) as dec(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(cr.cr_return_amount,0)) as dec(15,4))/
+ 		cast(sum(coalesce(cs.cs_net_paid,0)) as dec(15,4) )) as currency_ratio
+ 		from 
+ 		catalog_sales cs left outer join catalog_returns cr
+ 			on (cs.cs_order_number = cr.cr_order_number and 
+ 			cs.cs_item_sk = cr.cr_item_sk)
+                ,date_dim
+ 		where 
+ 			cr.cr_return_amount > 10000 
+ 			and cs.cs_net_profit > 1
+                         and cs.cs_net_paid > 0
+                         and cs.cs_quantity > 0
+                         and cs_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+                 group by cs.cs_item_sk
+ 	) in_cat
+ ) catalog
+ where 
+ (
+ catalog.return_rank <= 10
+ or
+ catalog.currency_rank <=10
+ )
+ union
+ select 
+ 'store' as channel
+ ,store.item
+ ,store.return_ratio
+ ,store.return_rank
+ ,store.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select sts.ss_item_sk as item
+ 		,(cast(sum(coalesce(sr.sr_return_quantity,0)) as dec(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as dec(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(sr.sr_return_amt,0)) as dec(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as dec(15,4) )) as currency_ratio
+ 		from 
+ 		store_sales sts left outer join store_returns sr
+ 			on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk)
+                ,date_dim
+ 		where 
+ 			sr.sr_return_amt > 10000 
+ 			and sts.ss_net_profit > 1
+                         and sts.ss_net_paid > 0 
+                         and sts.ss_quantity > 0
+                         and ss_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+ 		group by sts.ss_item_sk
+ 	) in_store
+ ) store
+ where  (
+ store.return_rank <= 10
+ or 
+ store.currency_rank <= 10
+ )
+ order by 1,4,5
+ limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 10 <- Union 9 (SIMPLE_EDGE)
+Reducer 11 <- Reducer 10 (SIMPLE_EDGE)
+Reducer 13 <- Map 12 (SIMPLE_EDGE), Map 24 (SIMPLE_EDGE)
+Reducer 14 <- Map 25 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE)
+Reducer 15 <- Reducer 14 (SIMPLE_EDGE)
+Reducer 16 <- Reducer 15 (SIMPLE_EDGE)
+Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Union 7 (CONTAINS)
+Reducer 18 <- Map 12 (SIMPLE_EDGE), Map 26 (SIMPLE_EDGE)
+Reducer 19 <- Map 27 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE)
+Reducer 20 <- Reducer 19 (SIMPLE_EDGE)
+Reducer 21 <- Reducer 20 (SIMPLE_EDGE)
+Reducer 22 <- Reducer 21 (SIMPLE_EDGE), Union 9 (CONTAINS)
+Reducer 3 <- Map 23 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE), Union 7 (CONTAINS)
+Reducer 8 <- Union 7 (SIMPLE_EDGE), Union 9 (CONTAINS)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 11
+      File Output Operator [FS_113]
+        Limit [LIM_112] (rows=100 width=101)
+          Number of rows:100
+          Select Operator [SEL_111] (rows=5915494 width=101)
+            Output:["_col0","_col1","_col2","_col3","_col4"]
+          <-Reducer 10 [SIMPLE_EDGE]
+            SHUFFLE [RS_110]
+              Select Operator [SEL_109] (rows=5915494 width=101)
+                Output:["_col0","_col1","_col2","_col3","_col4"]
+                Group By Operator [GBY_108] (rows=5915494 width=101)
+                  Output:["_col0","_col1","_col2","_col3","_col4"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
+                <-Union 9 [SIMPLE_EDGE]
+                  <-Reducer 22 [CONTAINS]
+                    Reduce Output Operator [RS_107]
+                      PartitionCols:_col0, _col1, _col2, _col3, _col4
+                      Group By Operator [GBY_106] (rows=11830988 width=101)
+                        Output:["_col0","_col1","_col2","_col3","_col4"],keys:_col0, _col3, _col4, _col1, _col2
+                        Select Operator [SEL_99] (rows=8604378 width=88)
+                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                          Filter Operator [FIL_137] (rows=8604378 width=88)
+                            predicate:((_col0 <= 10) or (rank_window_1 <= 10))
+                            PTF Operator [PTF_98] (rows=12906568 width=88)
+                              Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col4 AS decimal(15,4)) / CAST( _col5 AS decimal(15,4))) ASC NULLS FIRST","partition by:":"0"}]
+                              Select Operator [SEL_97] (rows=12906568 width=88)
+                                Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                              <-Reducer 21 [SIMPLE_EDGE]
+                                SHUFFLE [RS_96]
+                                  PartitionCols:0
+                                  Select Operator [SEL_95] (rows=12906568 width=88)
+                                    Output:["rank_window_0","_col0","_col1","_col2","_col3","_col4"]
+                                    PTF Operator [PTF_94] (rows=12906568 width=88)
+                                      Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col1 AS decimal(15,4)) / CAST( _col2 AS decimal(15,4))) ASC NULLS FIRST","partition by:":"0"}]
+                                      Select Operator [SEL_93] (rows=12906568 width=88)
+                                        Output:["_col0","_col1","_col2","_col3","_col4"]
+                                      <-Reducer 20 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_92]
+                                          PartitionCols:0
+                                          Group By Operator [GBY_90] (rows=12906568 width=88)
+                                            Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0
+                                          <-Reducer 19 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_89]
+                                              PartitionCols:_col0
+                                              Group By Operator [GBY_88] (rows=25813137 width=88)
+                                                Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col1)","sum(_col2)","sum(_col3)","sum(_col4)"],keys:_col0
+                                                Select Operator [SEL_86] (rows=25813137 width=88)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                  Merge Join Operator [MERGEJOIN_146] (rows=25813137 width=88)
+                                                    Conds:RS_83._col1, _col2=RS_84._col0, _col1(Inner),Output:["_col1","_col3","_col4","_col11","_col12"]
+                                                  <-Map 27 [SIMPLE_EDGE]
+                                                    SHUFFLE [RS_84]
+                                                      PartitionCols:_col0, _col1
+                                                      Select Operator [SEL_79] (rows=19197050 width=77)
+                                                        Output:["_col0","_col1","_col2","_col3"]
+                                                        Filter Operator [FIL_140] (rows=19197050 width=77)
+                                                          predicate:((sr_return_amt > 10000) and sr_item_sk is not null and sr_ticket_number is not null)
+                                                          TableScan [TS_77] (rows=57591150 width=77)
+                                                            default@store_returns,sr,Tbl:COMPLETE,Col:NONE,Output:["sr_item_sk","sr_ticket_number","sr_return_quantity","sr_return_amt"]
+                                                  <-Reducer 18 [SIMPLE_EDGE]
+                                                    SHUFFLE [RS_83]
+                                                      PartitionCols:_col1, _col2
+                                                      Merge Join Operator [MERGEJOIN_145] (rows=23466488 width=88)
+                                                        Conds:RS_80._col0=RS_81._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                                      <-Map 12 [SIMPLE_EDGE]
+                                                        SHUFFLE [RS_81]
+                                                          PartitionCols:_col0
+                                                          Select Operator [SEL_76] (rows=18262 width=1119)
+                                                            Output:["_col0"]
+                                                            Filter Operator [FIL_139] (rows=18262 width=1119)
+                                                              predicate:((d_year = 2000) and (d_moy = 12) and d_date_sk is not null)
+                                                              TableScan [TS_3] (rows=73049 width=1119)
+                                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
+                                                      <-Map 26 [SIMPLE_EDGE]
+                                                        SHUFFLE [RS_80]
+                                                          PartitionCols:_col0
+                                                          Select Operator [SEL_73] (rows=21333171 width=88)
+                                                            Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                            Filter Operator [FIL_138] (rows=21333171 width=88)
+                                                              predicate:((ss_net_profit > 1) and (ss_net_paid > 0) and (ss_quantity > 0) and ss_item_sk is not null and ss_ticket_number is not null and ss_sold_date_sk is not null)
+                                                              TableScan [TS_71] (rows=575995635 width=88)
+                                                                default@store_sales,sts,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ticket_number","ss_quantity","ss_net_paid","ss_net_profit"]
+                  <-Reducer 8 [CONTAINS]
+                    Reduce Output Operator [RS_107]
+                      PartitionCols:_col0, _col1, _col2, _col3, _col4
+                      Group By Operator [GBY_106] (rows=11830988 width=101)
+                        Output:["_col0","_col1","_col2","_col3","_col4"],keys:_col0, _col3, _col4, _col1, _col2
+                        Select Operator [SEL_70] (rows=3226610 width=135)
+                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                          Group By Operator [GBY_69] (rows=3226610 width=135)
+                            Output:["_col0","_col1","_col2","_col3","_col4"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
+                          <-Union 7 [SIMPLE_EDGE]
+                            <-Reducer 17 [CONTAINS]
+                              Reduce Output Operator [RS_68]
+                                PartitionCols:_col0, _col1, _col2, _col3, _col4
+                                Group By Operator [GBY_67] (rows=6453220 width=135)
+                                  Output:["_col0","_col1","_col2","_col3","_col4"],keys:_col0, _col3, _col4, _col1, _col2
+                                  Select Operator [SEL_60] (rows=4302070 width=135)
+                                    Output:["_col0","_col1","_col2","_col3","_col4"]
+                                    Filter Operator [FIL_133] (rows=4302070 width=135)
+                                      predicate:((_col0 <= 10) or (rank_window_1 <= 10))
+                                      PTF Operator [PTF_59] (rows=6453105 width=135)
+                                        Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col4 AS decimal(15,4)) / CAST( _col5 AS decimal(15,4))) ASC NULLS FIRST","partition by:":"0"}]
+                                        Select Operator [SEL_58] (rows=6453105 width=135)
+                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                        <-Reducer 16 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_57]
+                                            PartitionCols:0
+                                            Select Operator [SEL_56] (rows=6453105 width=135)
+                                              Output:["rank_window_0","_col0","_col1","_col2","_col3","_col4"]
+                                              PTF Operator [PTF_55] (rows=6453105 width=135)
+                                                Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col1 AS decimal(15,4)) / CAST( _col2 AS decimal(15,4))) ASC NULLS FIRST","partition by:":"0"}]
+                                                Select Operator [SEL_54] (rows=6453105 width=135)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                <-Reducer 15 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_53]
+                                                    PartitionCols:0
+                                                    Group By Operator [GBY_51] (rows=6453105 width=135)
+                                                      Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0
+                                                    <-Reducer 14 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_50]
+                                                        PartitionCols:_col0
+                                                        Group By Operator [GBY_49] (rows=12906211 width=135)
+                                                          Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col1)","sum(_col2)","sum(_col3)","sum(_col4)"],keys:_col0
+                                                          Select Operator [SEL_47] (rows=12906211 width=135)
+                                                            Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                            Merge Join Operator [MERGEJOIN_144] (rows=12906211 width=135)
+                                                              Conds:RS_44._col1, _col2=RS_45._col0, _col1(Inner),Output:["_col1","_col3","_col4","_col11","_col12"]
+                                                            <-Map 25 [SIMPLE_EDGE]
+                                                              SHUFFLE [RS_45]
+                                                                PartitionCols:_col0, _col1
+                                                                Select Operator [SEL_40] (rows=9599627 width=106)
+                                                                  Output:["_col0","_col1","_col2","_col3"]
+                                                                  Filter Operator [FIL_136] (rows=9599627 width=106)
+                                                                    predicate:((cr_return_amount > 10000) and cr_order_number is not null and cr_item_sk is not null)
+                                                                    TableScan [TS_38] (rows=28798881 width=106)
+                                                                      default@catalog_returns,cr,Tbl:COMPLETE,Col:NONE,Output:["cr_item_sk","cr_order_number","cr_return_quantity","cr_return_amount"]
+                                                            <-Reducer 13 [SIMPLE_EDGE]
+                                                              SHUFFLE [RS_44]
+                                                                PartitionCols:_col1, _col2
+                                                                Merge Join Operator [MERGEJOIN_143] (rows=11732919 width=135)
+                                                                  Conds:RS_41._col0=RS_42._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                                                <-Map 12 [SIMPLE_EDGE]
+                                                                  SHUFFLE [RS_42]
+                                                                    PartitionCols:_col0
+                                                                    Select Operator [SEL_37] (rows=18262 width=1119)
+                                                                      Output:["_col0"]
+                                                                      Filter Operator [FIL_135] (rows=18262 width=1119)
+                                                                        predicate:((d_year = 2000) and (d_moy = 12) and d_date_sk is not null)
+                                                                         Please refer to the previous TableScan [TS_3]
+                                                                <-Map 24 [SIMPLE_EDGE]
+                                                                  SHUFFLE [RS_41]
+                                                                    PartitionCols:_col0
+                                                                    Select Operator [SEL_34] (rows=10666290 width=135)
+                                                                      Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                                      Filter Operator [FIL_134] (rows=10666290 width=135)
+                                                                        predicate:((cs_net_profit > 1) and (cs_net_paid > 0) and (cs_quantity > 0) and cs_order_number is not null and cs_item_sk is not null and cs_sold_date_sk is not null)
+                                                                        TableScan [TS_32] (rows=287989836 width=135)
+                                                                          default@catalog_sales,cs,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_order_number","cs_quantity","cs_net_paid","cs_net_profit"]
+                            <-Reducer 6 [CONTAINS]
+                              Reduce Output Operator [RS_68]
+                                PartitionCols:_col0, _col1, _col2, _col3, _col4
+                                Group By Operator [GBY_67] (rows=6453220 width=135)
+                                  Output:["_col0","_col1","_col2","_col3","_col4"],keys:_col0, _col3, _col4, _col1, _col2
+                                  Select Operator [SEL_28] (rows=2151150 width=135)
+                                    Output:["_col0","_col1","_col2","_col3","_col4"]
+                                    Filter Operator [FIL_129] (rows=2151150 width=135)
+                                      predicate:((_col0 <= 10) or (rank_window_1 <= 10))
+                                      PTF Operator [PTF_27] (rows=3226726 width=135)
+                                        Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col4 AS decimal(15,4)) / CAST( _col5 AS decimal(15,4))) ASC NULLS FIRST","partition by:":"0"}]
+                                        Select Operator [SEL_26] (rows=3226726 width=135)
+                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                        <-Reducer 5 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_25]
+                                            PartitionCols:0
+                                            Select Operator [SEL_24] (rows=3226726 width=135)
+                                              Output:["rank_window_0","_col0","_col1","_col2","_col3","_col4"]
+                                              PTF Operator [PTF_23] (rows=3226726 width=135)
+                                                Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col1 AS decimal(15,4)) / CAST( _col2 AS decimal(15,4))) ASC NULLS FIRST","partition by:":"0"}]
+                                                Select Operator [SEL_22] (rows=3226726 width=135)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                <-Reducer 4 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_21]
+                                                    PartitionCols:0
+                                                    Group By Operator [GBY_19] (rows=3226726 width=135)
+                                                      Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0
+                                                    <-Reducer 3 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_18]
+                                                        PartitionCols:_col0
+                                                        Group By Operator [GBY_17] (rows=6453452 width=135)
+                                                          Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col1)","sum(_col2)","sum(_col3)","sum(_col4)"],keys:_col0
+                                                          Select Operator [SEL_15] (rows=6453452 width=135)
+                                                            Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                            Merge Join Operator [MERGEJOIN_142] (rows=6453452 width=135)
+                                                              Conds:RS_12._col1, _col2=RS_13._col0, _col1(Inner),Output:["_col1","_col3","_col4","_col11","_col12"]
+                                                            <-Map 23 [SIMPLE_EDGE]
+                                                              SHUFFLE [RS_13]
+                                                                PartitionCols:_col0, _col1
+                                                                Select Operator [SEL_8] (rows=4799489 width=92)
+                                                                  Output:["_col0","_col1","_col2","_col3"]
+                                                                  Filter Operator [FIL_132] (rows=4799489 width=92)
+                                                                    predicate:((wr_return_amt > 10000) and wr_item_sk is not null and wr_order_number is not null)
+                                                                    TableScan [TS_6] (rows=14398467 width=92)
+                                                                      default@web_returns,wr,Tbl:COMPLETE,Col:NONE,Output:["wr_item_sk","wr_order_number","wr_return_quantity","wr_return_amt"]
+                                                            <-Reducer 2 [SIMPLE_EDGE]
+                                                              SHUFFLE [RS_12]
+                                                                PartitionCols:_col1, _col2
+                                                                Merge Join Operator [MERGEJOIN_141] (rows=5866775 width=135)
+                                                                  Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                                                <-Map 12 [SIMPLE_EDGE]
+                                                                  SHUFFLE [RS_10]
+                                                                    PartitionCols:_col0
+                                                                    Select Operator [SEL_5] (rows=18262 width=1119)
+                                                                      Output:["_col0"]
+                                                                      Filter Operator [FIL_131] (rows=18262 width=1119)
+                                                                        predicate:((d_year = 2000) and (d_moy = 12) and d_date_sk is not null)
+                                                                         Please refer to the previous TableScan [TS_3]
+                                                                <-Map 1 [SIMPLE_EDGE]
+                                                                  SHUFFLE [RS_9]
+                                                                    PartitionCols:_col0
+                                                                    Select Operator [SEL_2] (rows=5333432 width=135)
+                                                                      Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                                      Filter Operator [FIL_130] (rows=5333432 width=135)
+                                                                        predicate:((ws_net_profit > 1) and (ws_net_paid > 0) and (ws_quantity > 0) and ws_order_number is not null and ws_item_sk is not null and ws_sold_date_sk is not null)
+                                                                        TableScan [TS_0] (rows=144002668 width=135)
+                                                                          default@web_sales,ws,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_order_number","ws_quantity","ws_net_paid","ws_net_profit"]
+

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query50.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query50.q.out b/ql/src/test/results/clientpositive/perf/query50.q.out
index 47a00b0..149dc98 100644
--- a/ql/src/test/results/clientpositive/perf/query50.q.out
+++ b/ql/src/test/results/clientpositive/perf/query50.q.out
@@ -1,4 +1,4 @@
-PREHOOK: query: explain 
+PREHOOK: query: explain
 select  
    s_store_name
   ,s_company_id
@@ -10,14 +10,14 @@ select
   ,s_county
   ,s_state
   ,s_zip
-  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end)  as 30days 
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end)  as `30 days` 
   ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and 
-                 (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end )  as 3160days 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end )  as `31-60 days` 
   ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and 
-                 (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end)  as 6190days 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end)  as `61-90 days` 
   ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and
-                 (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end)  as 91120days 
-  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk  > 120) then 1 else 0 end)  as 120days 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end)  as `91-120 days` 
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk  > 120) then 1 else 0 end)  as `>120 days` 
 from
    store_sales
   ,store_returns
@@ -27,12 +27,12 @@ from
 where
     d2.d_year = 2000
 and d2.d_moy  = 9
-and store_sales.ss_ticket_number = store_returns.sr_ticket_number
-and store_sales.ss_item_sk = store_returns.sr_item_sk
-and store_sales.ss_sold_date_sk   = d1.d_date_sk
+and ss_ticket_number = sr_ticket_number
+and ss_item_sk = sr_item_sk
+and ss_sold_date_sk   = d1.d_date_sk
 and sr_returned_date_sk   = d2.d_date_sk
-and store_sales.ss_customer_sk = store_returns.sr_customer_sk
-and store_sales.ss_store_sk = store.s_store_sk
+and ss_customer_sk = sr_customer_sk
+and ss_store_sk = s_store_sk
 group by
    s_store_name
   ,s_company_id
@@ -56,7 +56,7 @@ order by s_store_name
         ,s_zip
 limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain 
+POSTHOOK: query: explain
 select  
    s_store_name
   ,s_company_id
@@ -68,14 +68,14 @@ select
   ,s_county
   ,s_state
   ,s_zip
-  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end)  as 30days 
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end)  as `30 days` 
   ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and 
-                 (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end )  as 3160days 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end )  as `31-60 days` 
   ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and 
-                 (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end)  as 6190days 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end)  as `61-90 days` 
   ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and
-                 (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end)  as 91120days 
-  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk  > 120) then 1 else 0 end)  as 120days 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end)  as `91-120 days` 
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk  > 120) then 1 else 0 end)  as `>120 days` 
 from
    store_sales
   ,store_returns
@@ -85,12 +85,12 @@ from
 where
     d2.d_year = 2000
 and d2.d_moy  = 9
-and store_sales.ss_ticket_number = store_returns.sr_ticket_number
-and store_sales.ss_item_sk = store_returns.sr_item_sk
-and store_sales.ss_sold_date_sk   = d1.d_date_sk
+and ss_ticket_number = sr_ticket_number
+and ss_item_sk = sr_item_sk
+and ss_sold_date_sk   = d1.d_date_sk
 and sr_returned_date_sk   = d2.d_date_sk
-and store_sales.ss_customer_sk = store_returns.sr_customer_sk
-and store_sales.ss_store_sk = store.s_store_sk
+and ss_customer_sk = sr_customer_sk
+and ss_store_sk = s_store_sk
 group by
    s_store_name
   ,s_company_id

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query51.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query51.q.out b/ql/src/test/results/clientpositive/perf/query51.q.out
index 2468c77..0ce3e9f 100644
--- a/ql/src/test/results/clientpositive/perf/query51.q.out
+++ b/ql/src/test/results/clientpositive/perf/query51.q.out
@@ -1,23 +1,24 @@
-PREHOOK: query: explain WITH web_v1 as (
+PREHOOK: query: explain
+WITH web_v1 as (
 select
-  ws_item_sk item_sk, d_date, sum(ws_sales_price),
+  ws_item_sk item_sk, d_date,
   sum(sum(ws_sales_price))
       over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
 from web_sales
     ,date_dim
 where ws_sold_date_sk=d_date_sk
-  and d_month_seq between 1193 and 1193+11
+  and d_month_seq between 1212 and 1212+11
   and ws_item_sk is not NULL
 group by ws_item_sk, d_date),
 store_v1 as (
 select
-  ss_item_sk item_sk, d_date, sum(ss_sales_price),
+  ss_item_sk item_sk, d_date,
   sum(sum(ss_sales_price))
       over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
 from store_sales
     ,date_dim
 where ss_sold_date_sk=d_date_sk
-  and d_month_seq between 1193 and 1193+11
+  and d_month_seq between 1212 and 1212+11
   and ss_item_sk is not NULL
 group by ss_item_sk, d_date)
  select  *
@@ -41,26 +42,27 @@ order by item_sk
         ,d_date
 limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain WITH web_v1 as (
+POSTHOOK: query: explain
+WITH web_v1 as (
 select
-  ws_item_sk item_sk, d_date, sum(ws_sales_price),
+  ws_item_sk item_sk, d_date,
   sum(sum(ws_sales_price))
       over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
 from web_sales
     ,date_dim
 where ws_sold_date_sk=d_date_sk
-  and d_month_seq between 1193 and 1193+11
+  and d_month_seq between 1212 and 1212+11
   and ws_item_sk is not NULL
 group by ws_item_sk, d_date),
 store_v1 as (
 select
-  ss_item_sk item_sk, d_date, sum(ss_sales_price),
+  ss_item_sk item_sk, d_date,
   sum(sum(ss_sales_price))
       over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
 from store_sales
     ,date_dim
 where ss_sold_date_sk=d_date_sk
-  and d_month_seq between 1193 and 1193+11
+  and d_month_seq between 1212 and 1212+11
   and ss_item_sk is not NULL
 group by ss_item_sk, d_date)
  select  *
@@ -142,7 +144,7 @@ Stage-0
                                           Select Operator [SEL_5] (rows=8116 width=1119)
                                             Output:["_col0","_col1"]
                                             Filter Operator [FIL_60] (rows=8116 width=1119)
-                                              predicate:(d_month_seq BETWEEN 1193 AND 1204 and d_date_sk is not null)
+                                              predicate:(d_month_seq BETWEEN 1212 AND 1223 and d_date_sk is not null)
                                               TableScan [TS_3] (rows=73049 width=1119)
                                                 default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date","d_month_seq"]
                                       <-Map 1 [SIMPLE_EDGE]
@@ -176,7 +178,7 @@ Stage-0
                                           Select Operator [SEL_25] (rows=8116 width=1119)
                                             Output:["_col0","_col1"]
                                             Filter Operator [FIL_62] (rows=8116 width=1119)
-                                              predicate:(d_month_seq BETWEEN 1193 AND 1204 and d_date_sk is not null)
+                                              predicate:(d_month_seq BETWEEN 1212 AND 1223 and d_date_sk is not null)
                                                Please refer to the previous TableScan [TS_3]
                                       <-Map 10 [SIMPLE_EDGE]
                                         SHUFFLE [RS_26]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query52.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query52.q.out b/ql/src/test/results/clientpositive/perf/query52.q.out
index 3d4b9e5..9631f59 100644
--- a/ql/src/test/results/clientpositive/perf/query52.q.out
+++ b/ql/src/test/results/clientpositive/perf/query52.q.out
@@ -1,6 +1,44 @@
-PREHOOK: query: explain select dt.d_year ,item.i_brand_id brand_id ,item.i_brand brand ,sum(ss_ext_sales_price) ext_price from date_dim dt ,store_sales ,item where dt.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and item.i_manager_id = 1 and dt.d_moy=12 and dt.d_year=1998 group by dt.d_year ,item.i_brand ,item.i_brand_id order by dt.d_year ,ext_price desc ,brand_id limit 100
+PREHOOK: query: explain
+select  dt.d_year
+ 	,item.i_brand_id brand_id
+ 	,item.i_brand brand
+ 	,sum(ss_ext_sales_price) ext_price
+ from date_dim dt
+     ,store_sales
+     ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+    and store_sales.ss_item_sk = item.i_item_sk
+    and item.i_manager_id = 1
+    and dt.d_moy=12
+    and dt.d_year=1998
+ group by dt.d_year
+ 	,item.i_brand
+ 	,item.i_brand_id
+ order by dt.d_year
+ 	,ext_price desc
+ 	,brand_id
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select dt.d_year ,item.i_brand_id brand_id ,item.i_brand brand ,sum(ss_ext_sales_price) ext_price from date_dim dt ,store_sales ,item where dt.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and item.i_manager_id = 1 and dt.d_moy=12 and dt.d_year=1998 group by dt.d_year ,item.i_brand ,item.i_brand_id order by dt.d_year ,ext_price desc ,brand_id limit 100
+POSTHOOK: query: explain
+select  dt.d_year
+ 	,item.i_brand_id brand_id
+ 	,item.i_brand brand
+ 	,sum(ss_ext_sales_price) ext_price
+ from date_dim dt
+     ,store_sales
+     ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+    and store_sales.ss_item_sk = item.i_item_sk
+    and item.i_manager_id = 1
+    and dt.d_moy=12
+    and dt.d_year=1998
+ group by dt.d_year
+ 	,item.i_brand
+ 	,item.i_brand_id
+ order by dt.d_year
+ 	,ext_price desc
+ 	,brand_id
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query53.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query53.q.out b/ql/src/test/results/clientpositive/perf/query53.q.out
new file mode 100644
index 0000000..bc9e6c4
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query53.q.out
@@ -0,0 +1,141 @@
+PREHOOK: query: explain
+select  * from 
+(select i_manufact_id,
+sum(ss_sales_price) sum_sales,
+avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales
+from item, store_sales, date_dim, store
+where ss_item_sk = i_item_sk and
+ss_sold_date_sk = d_date_sk and
+ss_store_sk = s_store_sk and
+d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11) and
+((i_category in ('Books','Children','Electronics') and
+i_class in ('personal','portable','reference','self-help') and
+i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+		'exportiunivamalg #9','scholaramalgamalg #9'))
+or(i_category in ('Women','Music','Men') and
+i_class in ('accessories','classical','fragrances','pants') and
+i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+		'importoamalg #1')))
+group by i_manufact_id, d_qoy ) tmp1
+where case when avg_quarterly_sales > 0 
+	then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales 
+	else null end > 0.1
+order by avg_quarterly_sales,
+	 sum_sales,
+	 i_manufact_id
+limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select  * from 
+(select i_manufact_id,
+sum(ss_sales_price) sum_sales,
+avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales
+from item, store_sales, date_dim, store
+where ss_item_sk = i_item_sk and
+ss_sold_date_sk = d_date_sk and
+ss_store_sk = s_store_sk and
+d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11) and
+((i_category in ('Books','Children','Electronics') and
+i_class in ('personal','portable','reference','self-help') and
+i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+		'exportiunivamalg #9','scholaramalgamalg #9'))
+or(i_category in ('Women','Music','Men') and
+i_class in ('accessories','classical','fragrances','pants') and
+i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+		'importoamalg #1')))
+group by i_manufact_id, d_qoy ) tmp1
+where case when avg_quarterly_sales > 0 
+	then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales 
+	else null end > 0.1
+order by avg_quarterly_sales,
+	 sum_sales,
+	 i_manufact_id
+limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 6
+      File Output Operator [FS_36]
+        Limit [LIM_35] (rows=100 width=88)
+          Number of rows:100
+          Select Operator [SEL_34] (rows=191662559 width=88)
+            Output:["_col0","_col1","_col2"]
+          <-Reducer 5 [SIMPLE_EDGE]
+            SHUFFLE [RS_33]
+              Select Operator [SEL_30] (rows=191662559 width=88)
+                Output:["_col0","_col1","_col2"]
+                Filter Operator [FIL_46] (rows=191662559 width=88)
+                  predicate:CASE WHEN ((avg_window_0 > 0)) THEN (((abs((_col2 - avg_window_0)) / avg_window_0) > 0.1)) ELSE (null) END
+                  Select Operator [SEL_29] (rows=383325119 width=88)
+                    Output:["avg_window_0","_col0","_col2"]
+                    PTF Operator [PTF_28] (rows=383325119 width=88)
+                      Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col0 ASC NULLS FIRST","partition by:":"_col0"}]
+                      Select Operator [SEL_25] (rows=383325119 width=88)
+                        Output:["_col0","_col2"]
+                        Group By Operator [GBY_24] (rows=383325119 width=88)
+                          Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
+                        <-Reducer 4 [SIMPLE_EDGE]
+                          SHUFFLE [RS_23]
+                            PartitionCols:_col0
+                            Group By Operator [GBY_22] (rows=766650239 width=88)
+                              Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col8, _col11
+                              Merge Join Operator [MERGEJOIN_54] (rows=766650239 width=88)
+                                Conds:RS_18._col2=RS_19._col0(Inner),Output:["_col3","_col8","_col11"]
+                              <-Map 9 [SIMPLE_EDGE]
+                                SHUFFLE [RS_19]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_11] (rows=1704 width=1910)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_50] (rows=1704 width=1910)
+                                      predicate:s_store_sk is not null
+                                      TableScan [TS_9] (rows=1704 width=1910)
+                                        default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk"]
+                              <-Reducer 3 [SIMPLE_EDGE]
+                                SHUFFLE [RS_18]
+                                  PartitionCols:_col2
+                                  Merge Join Operator [MERGEJOIN_53] (rows=696954748 width=88)
+                                    Conds:RS_15._col0=RS_16._col0(Inner),Output:["_col2","_col3","_col8","_col11"]
+                                  <-Map 8 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_16]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_8] (rows=36525 width=1119)
+                                        Output:["_col0","_col2"]
+                                        Filter Operator [FIL_49] (rows=36525 width=1119)
+                                          predicate:((d_month_seq) IN (1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222, 1223) 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_month_seq","d_qoy"]
+                                  <-Reducer 2 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_15]
+                                      PartitionCols:_col0
+                                      Merge Join Operator [MERGEJOIN_52] (rows=633595212 width=88)
+                                        Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col0","_col2","_col3","_col8"]
+                                      <-Map 1 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_12]
+                                          PartitionCols:_col1
+                                          Select Operator [SEL_2] (rows=575995635 width=88)
+                                            Output:["_col0","_col1","_col2","_col3"]
+                                            Filter Operator [FIL_47] (rows=575995635 width=88)
+                                              predicate:(ss_item_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null)
+                                              TableScan [TS_0] (rows=575995635 width=88)
+                                                default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_store_sk","ss_sales_price"]
+                                      <-Map 7 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_13]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_5] (rows=115500 width=1436)
+                                            Output:["_col0","_col4"]
+                                            Filter Operator [FIL_48] (rows=115500 width=1436)
+                                              predicate:(((i_class) IN ('personal', 'portable', 'reference', 'self-help') or (i_class) IN ('accessories', 'classical', 'fragrances', 'pants')) and ((i_brand) IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9') or (i_brand) IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')) and ((i_category) IN ('Books', 'Children', 'Electronics') or (i_category) IN ('Women', 'Music', 'Men')) and (((i_category) IN ('Books', 'Children', 'Electronics') and (i_class) IN ('personal', 'portable', 'reference', 'self-help') and (i_brand) IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) or ((i_category) IN ('Women', 'Music', 'Men') and (i_class) IN ('accessories', 'classical', 'fragrances', 'pants') and (i_brand) IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_item_sk is not null)
+                                              TableScan [TS_3] (rows=462000 width=1436)
+                                                default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand","i_class","i_category","i_manufact_id"]
+


[13/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query13.q.out b/ql/src/test/results/clientpositive/perf/query13.q.out
index d67abe2..ac9d3cb 100644
--- a/ql/src/test/results/clientpositive/perf/query13.q.out
+++ b/ql/src/test/results/clientpositive/perf/query13.q.out
@@ -1,110 +1,102 @@
-PREHOOK: query: EXPLAIN SELECT Avg(ss_quantity) , 
-       Avg(ss_ext_sales_price) , 
-       Avg(ss_ext_wholesale_cost) , 
-       Sum(ss_ext_wholesale_cost) 
-FROM   store_sales , 
-       store , 
-       customer_demographics , 
-       household_demographics , 
-       customer_address , 
-       date_dim 
-WHERE  store.s_store_sk = store_sales.ss_store_sk 
-AND    store_sales.ss_sold_date_sk = date_dim.d_date_sk 
-AND    date_dim.d_year = 2001 
-AND   (( 
-                     store_sales.ss_hdemo_sk=household_demographics.hd_demo_sk 
-              AND    customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk 
-              AND    customer_demographics.cd_marital_status = 'M' 
-              AND    customer_demographics.cd_education_status = '4 yr Degree' 
-              AND    store_sales.ss_sales_price BETWEEN 100.00 AND    150.00 
-              AND    household_demographics.hd_dep_count = 3 ) 
-       OR     ( 
-                     store_sales.ss_hdemo_sk=household_demographics.hd_demo_sk 
-              AND    customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk 
-              AND    customer_demographics.cd_marital_status = 'D' 
-              AND    customer_demographics.cd_education_status = 'Primary' 
-              AND    store_sales.ss_sales_price BETWEEN 50.00 AND    100.00 
-              AND    household_demographics.hd_dep_count = 1 ) 
-       OR     ( 
-                     store_sales.ss_hdemo_sk=household_demographics.hd_demo_sk 
-              AND    customer_demographics.cd_demo_sk = ss_cdemo_sk 
-              AND    customer_demographics.cd_marital_status = 'U' 
-              AND    customer_demographics.cd_education_status = 'Advanced Degree' 
-              AND    store_sales.ss_sales_price BETWEEN 150.00 AND    200.00 
-              AND    household_demographics.hd_dep_count = 1 )) 
-AND   (( 
-                     store_sales.ss_addr_sk = customer_address.ca_address_sk 
-              AND    customer_address.ca_country = 'United States' 
-              AND    customer_address.ca_state IN ('KY', 
-                                                   'GA', 
-                                                   'NM') 
-              AND    store_sales.ss_net_profit BETWEEN 100 AND    200 ) 
-       OR     ( 
-                     store_sales.ss_addr_sk = customer_address.ca_address_sk 
-              AND    customer_address.ca_country = 'United States' 
-              AND    customer_address.ca_state IN ('MT', 
-                                                   'OR', 
-                                                   'IN') 
-              AND    store_sales.ss_net_profit BETWEEN 150 AND    300 ) 
-       OR     ( 
-                     store_sales.ss_addr_sk = customer_address.ca_address_sk 
-              AND    customer_address.ca_country = 'United States' 
-              AND    customer_address.ca_state IN ('WI', 'MO', 'WV') 
-              AND    store_sales.ss_net_profit BETWEEN 50 AND    250 ))
+PREHOOK: query: explain
+select avg(ss_quantity)
+       ,avg(ss_ext_sales_price)
+       ,avg(ss_ext_wholesale_cost)
+       ,sum(ss_ext_wholesale_cost)
+ from store_sales
+     ,store
+     ,customer_demographics
+     ,household_demographics
+     ,customer_address
+     ,date_dim
+ where s_store_sk = ss_store_sk
+ and  ss_sold_date_sk = d_date_sk and d_year = 2001
+ and((ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'M'
+  and cd_education_status = '4 yr Degree'
+  and ss_sales_price between 100.00 and 150.00
+  and hd_dep_count = 3   
+     )or
+     (ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'D'
+  and cd_education_status = 'Primary'
+  and ss_sales_price between 50.00 and 100.00   
+  and hd_dep_count = 1
+     ) or 
+     (ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'U'
+  and cd_education_status = 'Advanced Degree'
+  and ss_sales_price between 150.00 and 200.00 
+  and hd_dep_count = 1  
+     ))
+ and((ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('KY', 'GA', 'NM')
+  and ss_net_profit between 100 and 200  
+     ) or
+     (ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('MT', 'OR', 'IN')
+  and ss_net_profit between 150 and 300  
+     ) or
+     (ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('WI', 'MO', 'WV')
+  and ss_net_profit between 50 and 250  
+     ))
 PREHOOK: type: QUERY
-POSTHOOK: query: EXPLAIN SELECT Avg(ss_quantity) , 
-       Avg(ss_ext_sales_price) , 
-       Avg(ss_ext_wholesale_cost) , 
-       Sum(ss_ext_wholesale_cost) 
-FROM   store_sales , 
-       store , 
-       customer_demographics , 
-       household_demographics , 
-       customer_address , 
-       date_dim 
-WHERE  store.s_store_sk = store_sales.ss_store_sk 
-AND    store_sales.ss_sold_date_sk = date_dim.d_date_sk 
-AND    date_dim.d_year = 2001 
-AND   (( 
-                     store_sales.ss_hdemo_sk=household_demographics.hd_demo_sk 
-              AND    customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk 
-              AND    customer_demographics.cd_marital_status = 'M' 
-              AND    customer_demographics.cd_education_status = '4 yr Degree' 
-              AND    store_sales.ss_sales_price BETWEEN 100.00 AND    150.00 
-              AND    household_demographics.hd_dep_count = 3 ) 
-       OR     ( 
-                     store_sales.ss_hdemo_sk=household_demographics.hd_demo_sk 
-              AND    customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk 
-              AND    customer_demographics.cd_marital_status = 'D' 
-              AND    customer_demographics.cd_education_status = 'Primary' 
-              AND    store_sales.ss_sales_price BETWEEN 50.00 AND    100.00 
-              AND    household_demographics.hd_dep_count = 1 ) 
-       OR     ( 
-                     store_sales.ss_hdemo_sk=household_demographics.hd_demo_sk 
-              AND    customer_demographics.cd_demo_sk = ss_cdemo_sk 
-              AND    customer_demographics.cd_marital_status = 'U' 
-              AND    customer_demographics.cd_education_status = 'Advanced Degree' 
-              AND    store_sales.ss_sales_price BETWEEN 150.00 AND    200.00 
-              AND    household_demographics.hd_dep_count = 1 )) 
-AND   (( 
-                     store_sales.ss_addr_sk = customer_address.ca_address_sk 
-              AND    customer_address.ca_country = 'United States' 
-              AND    customer_address.ca_state IN ('KY', 
-                                                   'GA', 
-                                                   'NM') 
-              AND    store_sales.ss_net_profit BETWEEN 100 AND    200 ) 
-       OR     ( 
-                     store_sales.ss_addr_sk = customer_address.ca_address_sk 
-              AND    customer_address.ca_country = 'United States' 
-              AND    customer_address.ca_state IN ('MT', 
-                                                   'OR', 
-                                                   'IN') 
-              AND    store_sales.ss_net_profit BETWEEN 150 AND    300 ) 
-       OR     ( 
-                     store_sales.ss_addr_sk = customer_address.ca_address_sk 
-              AND    customer_address.ca_country = 'United States' 
-              AND    customer_address.ca_state IN ('WI', 'MO', 'WV') 
-              AND    store_sales.ss_net_profit BETWEEN 50 AND    250 ))
+POSTHOOK: query: explain
+select avg(ss_quantity)
+       ,avg(ss_ext_sales_price)
+       ,avg(ss_ext_wholesale_cost)
+       ,sum(ss_ext_wholesale_cost)
+ from store_sales
+     ,store
+     ,customer_demographics
+     ,household_demographics
+     ,customer_address
+     ,date_dim
+ where s_store_sk = ss_store_sk
+ and  ss_sold_date_sk = d_date_sk and d_year = 2001
+ and((ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'M'
+  and cd_education_status = '4 yr Degree'
+  and ss_sales_price between 100.00 and 150.00
+  and hd_dep_count = 3   
+     )or
+     (ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'D'
+  and cd_education_status = 'Primary'
+  and ss_sales_price between 50.00 and 100.00   
+  and hd_dep_count = 1
+     ) or 
+     (ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'U'
+  and cd_education_status = 'Advanced Degree'
+  and ss_sales_price between 150.00 and 200.00 
+  and hd_dep_count = 1  
+     ))
+ and((ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('KY', 'GA', 'NM')
+  and ss_net_profit between 100 and 200  
+     ) or
+     (ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('MT', 'OR', 'IN')
+  and ss_net_profit between 150 and 300  
+     ) or
+     (ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('WI', 'MO', 'WV')
+  and ss_net_profit between 50 and 250  
+     ))
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query15.q.out b/ql/src/test/results/clientpositive/perf/query15.q.out
index adec728..a2d95f6 100644
--- a/ql/src/test/results/clientpositive/perf/query15.q.out
+++ b/ql/src/test/results/clientpositive/perf/query15.q.out
@@ -1,6 +1,40 @@
-PREHOOK: query: explain select ca_zip ,sum(cs_sales_price) from catalog_sales ,customer ,customer_address ,date_dim where catalog_sales.cs_bill_customer_sk = customer.c_customer_sk and customer.c_current_addr_sk = customer_address.ca_address_sk and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') or customer_address.ca_state in ('CA','WA','GA') or catalog_sales.cs_sales_price > 500) and catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and date_dim.d_qoy = 2 and date_dim.d_year = 2000 group by ca_zip order by ca_zip limit 100
+PREHOOK: query: explain
+select  ca_zip
+       ,sum(cs_sales_price)
+ from catalog_sales
+     ,customer
+     ,customer_address
+     ,date_dim
+ where cs_bill_customer_sk = c_customer_sk
+ 	and c_current_addr_sk = ca_address_sk 
+ 	and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475',
+                                   '85392', '85460', '80348', '81792')
+ 	      or ca_state in ('CA','WA','GA')
+ 	      or cs_sales_price > 500)
+ 	and cs_sold_date_sk = d_date_sk
+ 	and d_qoy = 2 and d_year = 2000
+ group by ca_zip
+ order by ca_zip
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select ca_zip ,sum(cs_sales_price) from catalog_sales ,customer ,customer_address ,date_dim where catalog_sales.cs_bill_customer_sk = customer.c_customer_sk and customer.c_current_addr_sk = customer_address.ca_address_sk and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') or customer_address.ca_state in ('CA','WA','GA') or catalog_sales.cs_sales_price > 500) and catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and date_dim.d_qoy = 2 and date_dim.d_year = 2000 group by ca_zip order by ca_zip limit 100
+POSTHOOK: query: explain
+select  ca_zip
+       ,sum(cs_sales_price)
+ from catalog_sales
+     ,customer
+     ,customer_address
+     ,date_dim
+ where cs_bill_customer_sk = c_customer_sk
+ 	and c_current_addr_sk = ca_address_sk 
+ 	and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475',
+                                   '85392', '85460', '80348', '81792')
+ 	      or ca_state in ('CA','WA','GA')
+ 	      or cs_sales_price > 500)
+ 	and cs_sold_date_sk = d_date_sk
+ 	and d_qoy = 2 and d_year = 2000
+ group by ca_zip
+ order by ca_zip
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query16.q.out b/ql/src/test/results/clientpositive/perf/query16.q.out
index cf90c0c..449b9c8 100644
--- a/ql/src/test/results/clientpositive/perf/query16.q.out
+++ b/ql/src/test/results/clientpositive/perf/query16.q.out
@@ -1,5 +1,6 @@
-Warning: Shuffle Join MERGEJOIN[106][tables = [$hdt$_2, $hdt$_3, $hdt$_1, $hdt$_4]] in Stage 'Reducer 17' is a cross product
-PREHOOK: query: explain select  
+Warning: Shuffle Join MERGEJOIN[107][tables = [$hdt$_2, $hdt$_3, $hdt$_1, $hdt$_4]] in Stage 'Reducer 17' is a cross product
+PREHOOK: query: explain
+select  
    count(distinct cs_order_number) as `order count`
   ,sum(cs_ext_ship_cost) as `total shipping cost`
   ,sum(cs_net_profit) as `total net profit`
@@ -25,10 +26,11 @@ and exists (select *
 and not exists(select *
                from catalog_returns cr1
                where cs1.cs_order_number = cr1.cr_order_number)
-order by `order count`
+order by count(distinct cs_order_number)
 limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select  
+POSTHOOK: query: explain
+select  
    count(distinct cs_order_number) as `order count`
   ,sum(cs_ext_ship_cost) as `total shipping cost`
   ,sum(cs_net_profit) as `total net profit`
@@ -54,7 +56,7 @@ and exists (select *
 and not exists(select *
                from catalog_returns cr1
                where cs1.cs_order_number = cr1.cr_order_number)
-order by `order count`
+order by count(distinct cs_order_number)
 limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
@@ -75,157 +77,159 @@ Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
       Reducer 8
-      File Output Operator [FS_73]
+      File Output Operator [FS_74]
         Limit [LIM_72] (rows=1 width=344)
           Number of rows:100
           Select Operator [SEL_71] (rows=1 width=344)
             Output:["_col0","_col1","_col2"]
           <-Reducer 7 [SIMPLE_EDGE]
             SHUFFLE [RS_70]
-              Group By Operator [GBY_68] (rows=1 width=344)
-                Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT KEY._col0:0._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
-              <-Reducer 6 [SIMPLE_EDGE]
-                SHUFFLE [RS_67]
-                  Group By Operator [GBY_66] (rows=1395035081047425024 width=1)
-                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT _col4)","sum(_col5)","sum(_col6)"],keys:_col4
-                    Select Operator [SEL_65] (rows=1395035081047425024 width=1)
-                      Output:["_col4","_col5","_col6"]
-                      Filter Operator [FIL_64] (rows=1395035081047425024 width=1)
-                        predicate:_col16 is null
-                        Select Operator [SEL_63] (rows=2790070162094850048 width=1)
-                          Output:["_col4","_col5","_col6","_col16"]
-                          Merge Join Operator [MERGEJOIN_112] (rows=2790070162094850048 width=1)
-                            Conds:RS_60._col3, _col4=RS_61._col0, _col1(Inner),Output:["_col4","_col5","_col6","_col14"]
-                          <-Reducer 16 [SIMPLE_EDGE]
-                            SHUFFLE [RS_61]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_46] (rows=2536427365110644736 width=1)
-                                Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                              <-Reducer 15 [SIMPLE_EDGE]
-                                SHUFFLE [RS_45]
-                                  PartitionCols:_col0, _col1
-                                  Group By Operator [GBY_44] (rows=5072854730221289472 width=1)
-                                    Output:["_col0","_col1"],keys:_col2, _col3
-                                    Select Operator [SEL_43] (rows=5072854730221289472 width=1)
-                                      Output:["_col2","_col3"]
-                                      Filter Operator [FIL_42] (rows=5072854730221289472 width=1)
-                                        predicate:(_col2 <> _col0)
-                                        Merge Join Operator [MERGEJOIN_110] (rows=5072854730221289472 width=1)
-                                          Conds:RS_39._col1=RS_40._col1(Inner),Output:["_col0","_col2","_col3"]
-                                        <-Map 14 [SIMPLE_EDGE]
-                                          PARTITION_ONLY_SHUFFLE [RS_39]
-                                            PartitionCols:_col1
-                                            Select Operator [SEL_20] (rows=287989836 width=135)
-                                              Output:["_col0","_col1"]
-                                              TableScan [TS_19] (rows=287989836 width=135)
-                                                default@catalog_sales,cs2,Tbl:COMPLETE,Col:NONE,Output:["cs_warehouse_sk","cs_order_number"]
-                                        <-Reducer 18 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_40]
-                                            PartitionCols:_col1
-                                            Select Operator [SEL_38] (rows=4611686018427387903 width=1)
-                                              Output:["_col0","_col1"]
-                                              Group By Operator [GBY_37] (rows=4611686018427387903 width=1)
-                                                Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                              <-Reducer 17 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_36]
-                                                  PartitionCols:_col0, _col1
-                                                  Group By Operator [GBY_35] (rows=9223372036854775807 width=1)
-                                                    Output:["_col0","_col1"],keys:_col4, _col3
-                                                    Merge Join Operator [MERGEJOIN_106] (rows=9223372036854775807 width=1)
-                                                      Conds:(Inner),(Inner),(Inner),Output:["_col3","_col4"]
-                                                    <-Map 14 [CUSTOM_SIMPLE_EDGE]
-                                                      PARTITION_ONLY_SHUFFLE [RS_32]
-                                                        Select Operator [SEL_28] (rows=287989836 width=135)
-                                                          Output:["_col0","_col1"]
-                                                           Please refer to the previous TableScan [TS_19]
-                                                    <-Map 19 [CUSTOM_SIMPLE_EDGE]
-                                                      PARTITION_ONLY_SHUFFLE [RS_29]
-                                                        Select Operator [SEL_22] (rows=73049 width=4)
-                                                          TableScan [TS_21] (rows=73049 width=1119)
-                                                            default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE
-                                                    <-Map 20 [CUSTOM_SIMPLE_EDGE]
-                                                      PARTITION_ONLY_SHUFFLE [RS_30]
-                                                        Select Operator [SEL_24] (rows=60 width=4)
-                                                          TableScan [TS_23] (rows=60 width=2045)
-                                                            default@call_center,call_center,Tbl:COMPLETE,Col:COMPLETE
-                                                    <-Map 21 [CUSTOM_SIMPLE_EDGE]
-                                                      PARTITION_ONLY_SHUFFLE [RS_31]
-                                                        Select Operator [SEL_26] (rows=40000000 width=4)
-                                                          TableScan [TS_25] (rows=40000000 width=1014)
-                                                            default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE
-                          <-Reducer 5 [SIMPLE_EDGE]
-                            SHUFFLE [RS_60]
-                              PartitionCols:_col3, _col4
-                              Merge Join Operator [MERGEJOIN_111] (rows=421645953 width=135)
-                                Conds:RS_57._col4=RS_58._col0(Left Outer),Output:["_col3","_col4","_col5","_col6","_col14"]
-                              <-Reducer 13 [SIMPLE_EDGE]
-                                SHUFFLE [RS_58]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_18] (rows=14399440 width=106)
-                                    Output:["_col0","_col1"]
-                                    Group By Operator [GBY_17] (rows=14399440 width=106)
-                                      Output:["_col0"],keys:KEY._col0
-                                    <-Map 12 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_16]
-                                        PartitionCols:_col0
-                                        Group By Operator [GBY_15] (rows=28798881 width=106)
-                                          Output:["_col0"],keys:cr_order_number
-                                          Filter Operator [FIL_103] (rows=28798881 width=106)
-                                            predicate:cr_order_number is not null
-                                            TableScan [TS_12] (rows=28798881 width=106)
-                                              default@catalog_returns,cr1,Tbl:COMPLETE,Col:NONE,Output:["cr_order_number"]
-                              <-Reducer 4 [SIMPLE_EDGE]
-                                SHUFFLE [RS_57]
-                                  PartitionCols:_col4
-                                  Merge Join Operator [MERGEJOIN_109] (rows=383314495 width=135)
-                                    Conds:RS_54._col2=RS_55._col0(Inner),Output:["_col3","_col4","_col5","_col6"]
-                                  <-Map 11 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_55]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_11] (rows=30 width=2045)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_102] (rows=30 width=2045)
-                                          predicate:((cc_county) IN ('Ziebach County', 'Levy County', 'Huron County', 'Franklin Parish', 'Daviess County') and cc_call_center_sk is not null)
-                                          TableScan [TS_9] (rows=60 width=2045)
-                                            default@call_center,call_center,Tbl:COMPLETE,Col:NONE,Output:["cc_call_center_sk","cc_county"]
-                                  <-Reducer 3 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_54]
-                                      PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_108] (rows=348467716 width=135)
-                                        Conds:RS_51._col1=RS_52._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col6"]
-                                      <-Map 10 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_52]
+              Select Operator [SEL_69] (rows=1 width=344)
+                Output:["_col1","_col2","_col3"]
+                Group By Operator [GBY_68] (rows=1 width=344)
+                  Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT KEY._col0:0._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
+                <-Reducer 6 [SIMPLE_EDGE]
+                  SHUFFLE [RS_67]
+                    Group By Operator [GBY_66] (rows=1395035081047425024 width=1)
+                      Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT _col4)","sum(_col5)","sum(_col6)"],keys:_col4
+                      Select Operator [SEL_65] (rows=1395035081047425024 width=1)
+                        Output:["_col4","_col5","_col6"]
+                        Filter Operator [FIL_64] (rows=1395035081047425024 width=1)
+                          predicate:_col16 is null
+                          Select Operator [SEL_63] (rows=2790070162094850048 width=1)
+                            Output:["_col4","_col5","_col6","_col16"]
+                            Merge Join Operator [MERGEJOIN_113] (rows=2790070162094850048 width=1)
+                              Conds:RS_60._col3, _col4=RS_61._col0, _col1(Inner),Output:["_col4","_col5","_col6","_col14"]
+                            <-Reducer 16 [SIMPLE_EDGE]
+                              SHUFFLE [RS_61]
+                                PartitionCols:_col0, _col1
+                                Group By Operator [GBY_46] (rows=2536427365110644736 width=1)
+                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                <-Reducer 15 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_45]
+                                    PartitionCols:_col0, _col1
+                                    Group By Operator [GBY_44] (rows=5072854730221289472 width=1)
+                                      Output:["_col0","_col1"],keys:_col2, _col3
+                                      Select Operator [SEL_43] (rows=5072854730221289472 width=1)
+                                        Output:["_col2","_col3"]
+                                        Filter Operator [FIL_42] (rows=5072854730221289472 width=1)
+                                          predicate:(_col2 <> _col0)
+                                          Merge Join Operator [MERGEJOIN_111] (rows=5072854730221289472 width=1)
+                                            Conds:RS_39._col1=RS_40._col1(Inner),Output:["_col0","_col2","_col3"]
+                                          <-Map 14 [SIMPLE_EDGE]
+                                            PARTITION_ONLY_SHUFFLE [RS_39]
+                                              PartitionCols:_col1
+                                              Select Operator [SEL_20] (rows=287989836 width=135)
+                                                Output:["_col0","_col1"]
+                                                TableScan [TS_19] (rows=287989836 width=135)
+                                                  default@catalog_sales,cs2,Tbl:COMPLETE,Col:NONE,Output:["cs_warehouse_sk","cs_order_number"]
+                                          <-Reducer 18 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_40]
+                                              PartitionCols:_col1
+                                              Select Operator [SEL_38] (rows=4611686018427387903 width=1)
+                                                Output:["_col0","_col1"]
+                                                Group By Operator [GBY_37] (rows=4611686018427387903 width=1)
+                                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                                <-Reducer 17 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_36]
+                                                    PartitionCols:_col0, _col1
+                                                    Group By Operator [GBY_35] (rows=9223372036854775807 width=1)
+                                                      Output:["_col0","_col1"],keys:_col4, _col3
+                                                      Merge Join Operator [MERGEJOIN_107] (rows=9223372036854775807 width=1)
+                                                        Conds:(Inner),(Inner),(Inner),Output:["_col3","_col4"]
+                                                      <-Map 14 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_32]
+                                                          Select Operator [SEL_28] (rows=287989836 width=135)
+                                                            Output:["_col0","_col1"]
+                                                             Please refer to the previous TableScan [TS_19]
+                                                      <-Map 19 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_29]
+                                                          Select Operator [SEL_22] (rows=73049 width=4)
+                                                            TableScan [TS_21] (rows=73049 width=1119)
+                                                              default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE
+                                                      <-Map 20 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_30]
+                                                          Select Operator [SEL_24] (rows=60 width=4)
+                                                            TableScan [TS_23] (rows=60 width=2045)
+                                                              default@call_center,call_center,Tbl:COMPLETE,Col:COMPLETE
+                                                      <-Map 21 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_31]
+                                                          Select Operator [SEL_26] (rows=40000000 width=4)
+                                                            TableScan [TS_25] (rows=40000000 width=1014)
+                                                              default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE
+                            <-Reducer 5 [SIMPLE_EDGE]
+                              SHUFFLE [RS_60]
+                                PartitionCols:_col3, _col4
+                                Merge Join Operator [MERGEJOIN_112] (rows=421645953 width=135)
+                                  Conds:RS_57._col4=RS_58._col0(Left Outer),Output:["_col3","_col4","_col5","_col6","_col14"]
+                                <-Reducer 13 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_58]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_18] (rows=14399440 width=106)
+                                      Output:["_col0","_col1"]
+                                      Group By Operator [GBY_17] (rows=14399440 width=106)
+                                        Output:["_col0"],keys:KEY._col0
+                                      <-Map 12 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_16]
                                           PartitionCols:_col0
-                                          Select Operator [SEL_8] (rows=20000000 width=1014)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_101] (rows=20000000 width=1014)
-                                              predicate:((ca_state = 'NY') and ca_address_sk is not null)
-                                              TableScan [TS_6] (rows=40000000 width=1014)
-                                                default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_state"]
-                                      <-Reducer 2 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_51]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_107] (rows=316788826 width=135)
-                                            Conds:RS_48._col0=RS_49._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
-                                          <-Map 1 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_48]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_2] (rows=287989836 width=135)
-                                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                Filter Operator [FIL_99] (rows=287989836 width=135)
-                                                  predicate:(cs_ship_date_sk is not null and cs_ship_addr_sk is not null and cs_call_center_sk is not null)
-                                                  TableScan [TS_0] (rows=287989836 width=135)
-                                                    default@catalog_sales,cs1,Tbl:COMPLETE,Col:NONE,Output:["cs_ship_date_sk","cs_ship_addr_sk","cs_call_center_sk","cs_warehouse_sk","cs_order_number","cs_ext_ship_cost","cs_net_profit"]
-                                          <-Map 9 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_49]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_5] (rows=8116 width=1119)
-                                                Output:["_col0"]
-                                                Filter Operator [FIL_100] (rows=8116 width=1119)
-                                                  predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 2001-04-01 00:00:00.0 AND 2001-05-31 01:00:00.0 and d_date_sk is not null)
-                                                  TableScan [TS_3] (rows=73049 width=1119)
-                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
+                                          Group By Operator [GBY_15] (rows=28798881 width=106)
+                                            Output:["_col0"],keys:cr_order_number
+                                            Filter Operator [FIL_104] (rows=28798881 width=106)
+                                              predicate:cr_order_number is not null
+                                              TableScan [TS_12] (rows=28798881 width=106)
+                                                default@catalog_returns,cr1,Tbl:COMPLETE,Col:NONE,Output:["cr_order_number"]
+                                <-Reducer 4 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_57]
+                                    PartitionCols:_col4
+                                    Merge Join Operator [MERGEJOIN_110] (rows=383314495 width=135)
+                                      Conds:RS_54._col2=RS_55._col0(Inner),Output:["_col3","_col4","_col5","_col6"]
+                                    <-Map 11 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_55]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_11] (rows=30 width=2045)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_103] (rows=30 width=2045)
+                                            predicate:((cc_county) IN ('Ziebach County', 'Levy County', 'Huron County', 'Franklin Parish', 'Daviess County') and cc_call_center_sk is not null)
+                                            TableScan [TS_9] (rows=60 width=2045)
+                                              default@call_center,call_center,Tbl:COMPLETE,Col:NONE,Output:["cc_call_center_sk","cc_county"]
+                                    <-Reducer 3 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_54]
+                                        PartitionCols:_col2
+                                        Merge Join Operator [MERGEJOIN_109] (rows=348467716 width=135)
+                                          Conds:RS_51._col1=RS_52._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col6"]
+                                        <-Map 10 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_52]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_8] (rows=20000000 width=1014)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_102] (rows=20000000 width=1014)
+                                                predicate:((ca_state = 'NY') and ca_address_sk is not null)
+                                                TableScan [TS_6] (rows=40000000 width=1014)
+                                                  default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_state"]
+                                        <-Reducer 2 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_51]
+                                            PartitionCols:_col1
+                                            Merge Join Operator [MERGEJOIN_108] (rows=316788826 width=135)
+                                              Conds:RS_48._col0=RS_49._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
+                                            <-Map 1 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_48]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_2] (rows=287989836 width=135)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                  Filter Operator [FIL_100] (rows=287989836 width=135)
+                                                    predicate:(cs_ship_date_sk is not null and cs_ship_addr_sk is not null and cs_call_center_sk is not null)
+                                                    TableScan [TS_0] (rows=287989836 width=135)
+                                                      default@catalog_sales,cs1,Tbl:COMPLETE,Col:NONE,Output:["cs_ship_date_sk","cs_ship_addr_sk","cs_call_center_sk","cs_warehouse_sk","cs_order_number","cs_ext_ship_cost","cs_net_profit"]
+                                            <-Map 9 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_49]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_5] (rows=8116 width=1119)
+                                                  Output:["_col0"]
+                                                  Filter Operator [FIL_101] (rows=8116 width=1119)
+                                                    predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 2001-04-01 00:00:00.0 AND 2001-05-31 01:00:00.0 and d_date_sk is not null)
+                                                    TableScan [TS_3] (rows=73049 width=1119)
+                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query17.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query17.q.out b/ql/src/test/results/clientpositive/perf/query17.q.out
index 2f32a9f..5b1f2a8 100644
--- a/ql/src/test/results/clientpositive/perf/query17.q.out
+++ b/ql/src/test/results/clientpositive/perf/query17.q.out
@@ -1,6 +1,90 @@
-PREHOOK: query: explain select i_item_id ,i_item_desc ,s_state ,count(ss_quantity) as store_sales_quantitycount ,avg(ss_quantity) as store_sales_quantityave ,stddev_samp(ss_quantity) as store_sales_quantitystdev ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov ,count(sr_return_quantity) as_store_returns_quantitycount ,avg(sr_return_quantity) as_store_returns_quantityave ,stddev_samp(sr_return_quantity) as_store_returns_quantitystdev ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitystdev ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov from store_sales ,store_returns ,catalog_sales ,date_dim d1 ,date_dim d2 ,date_dim d3 ,store ,item where d1.d_quarter_name = '2000Q1' and d1.d_date_sk = store_sales.ss_sold_date_sk and item.i_item_sk = store_s
 ales.ss_item_sk and store.s_store_sk = store_sales.ss_store_sk and store_sales.ss_customer_sk = store_returns.sr_customer_sk and store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number and store_returns.sr_returned_date_sk = d2.d_date_sk and d2.d_quarter_name in ('2000Q1','2000Q2','2000Q3') and store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk and store_returns.sr_item_sk = catalog_sales.cs_item_sk and catalog_sales.cs_sold_date_sk = d3.d_date_sk and d3.d_quarter_name in ('2000Q1','2000Q2','2000Q3') group by i_item_id ,i_item_desc ,s_state order by i_item_id ,i_item_desc ,s_state limit 100
+PREHOOK: query: explain
+select  i_item_id
+       ,i_item_desc
+       ,s_state
+       ,count(ss_quantity) as store_sales_quantitycount
+       ,avg(ss_quantity) as store_sales_quantityave
+       ,stddev_samp(ss_quantity) as store_sales_quantitystdev
+       ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov
+       ,count(sr_return_quantity) as_store_returns_quantitycount
+       ,avg(sr_return_quantity) as_store_returns_quantityave
+       ,stddev_samp(sr_return_quantity) as_store_returns_quantitystdev
+       ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov
+       ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave
+       ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitystdev
+       ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov
+ from store_sales
+     ,store_returns
+     ,catalog_sales
+     ,date_dim d1
+     ,date_dim d2
+     ,date_dim d3
+     ,store
+     ,item
+ where d1.d_quarter_name = '2000Q1'
+   and d1.d_date_sk = ss_sold_date_sk
+   and i_item_sk = ss_item_sk
+   and s_store_sk = ss_store_sk
+   and ss_customer_sk = sr_customer_sk
+   and ss_item_sk = sr_item_sk
+   and ss_ticket_number = sr_ticket_number
+   and sr_returned_date_sk = d2.d_date_sk
+   and d2.d_quarter_name in ('2000Q1','2000Q2','2000Q3')
+   and sr_customer_sk = cs_bill_customer_sk
+   and sr_item_sk = cs_item_sk
+   and cs_sold_date_sk = d3.d_date_sk
+   and d3.d_quarter_name in ('2000Q1','2000Q2','2000Q3')
+ group by i_item_id
+         ,i_item_desc
+         ,s_state
+ order by i_item_id
+         ,i_item_desc
+         ,s_state
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select i_item_id ,i_item_desc ,s_state ,count(ss_quantity) as store_sales_quantitycount ,avg(ss_quantity) as store_sales_quantityave ,stddev_samp(ss_quantity) as store_sales_quantitystdev ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov ,count(sr_return_quantity) as_store_returns_quantitycount ,avg(sr_return_quantity) as_store_returns_quantityave ,stddev_samp(sr_return_quantity) as_store_returns_quantitystdev ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitystdev ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov from store_sales ,store_returns ,catalog_sales ,date_dim d1 ,date_dim d2 ,date_dim d3 ,store ,item where d1.d_quarter_name = '2000Q1' and d1.d_date_sk = store_sales.ss_sold_date_sk and item.i_item_sk = store_
 sales.ss_item_sk and store.s_store_sk = store_sales.ss_store_sk and store_sales.ss_customer_sk = store_returns.sr_customer_sk and store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number and store_returns.sr_returned_date_sk = d2.d_date_sk and d2.d_quarter_name in ('2000Q1','2000Q2','2000Q3') and store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk and store_returns.sr_item_sk = catalog_sales.cs_item_sk and catalog_sales.cs_sold_date_sk = d3.d_date_sk and d3.d_quarter_name in ('2000Q1','2000Q2','2000Q3') group by i_item_id ,i_item_desc ,s_state order by i_item_id ,i_item_desc ,s_state limit 100
+POSTHOOK: query: explain
+select  i_item_id
+       ,i_item_desc
+       ,s_state
+       ,count(ss_quantity) as store_sales_quantitycount
+       ,avg(ss_quantity) as store_sales_quantityave
+       ,stddev_samp(ss_quantity) as store_sales_quantitystdev
+       ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov
+       ,count(sr_return_quantity) as_store_returns_quantitycount
+       ,avg(sr_return_quantity) as_store_returns_quantityave
+       ,stddev_samp(sr_return_quantity) as_store_returns_quantitystdev
+       ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov
+       ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave
+       ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitystdev
+       ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov
+ from store_sales
+     ,store_returns
+     ,catalog_sales
+     ,date_dim d1
+     ,date_dim d2
+     ,date_dim d3
+     ,store
+     ,item
+ where d1.d_quarter_name = '2000Q1'
+   and d1.d_date_sk = ss_sold_date_sk
+   and i_item_sk = ss_item_sk
+   and s_store_sk = ss_store_sk
+   and ss_customer_sk = sr_customer_sk
+   and ss_item_sk = sr_item_sk
+   and ss_ticket_number = sr_ticket_number
+   and sr_returned_date_sk = d2.d_date_sk
+   and d2.d_quarter_name in ('2000Q1','2000Q2','2000Q3')
+   and sr_customer_sk = cs_bill_customer_sk
+   and sr_item_sk = cs_item_sk
+   and cs_sold_date_sk = d3.d_date_sk
+   and d3.d_quarter_name in ('2000Q1','2000Q2','2000Q3')
+ group by i_item_id
+         ,i_item_desc
+         ,s_state
+ order by i_item_id
+         ,i_item_desc
+         ,s_state
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query18.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query18.q.out b/ql/src/test/results/clientpositive/perf/query18.q.out
index da35ca0..2ac6d4b 100644
--- a/ql/src/test/results/clientpositive/perf/query18.q.out
+++ b/ql/src/test/results/clientpositive/perf/query18.q.out
@@ -1,6 +1,68 @@
-PREHOOK: query: explain select i_item_id, ca_country, ca_state, ca_county, avg( cast(cs_quantity as decimal(12,2))) agg1, avg( cast(cs_list_price as decimal(12,2))) agg2, avg( cast(cs_coupon_amt as decimal(12,2))) agg3, avg( cast(cs_sales_price as decimal(12,2))) agg4, avg( cast(cs_net_profit as decimal(12,2))) agg5, avg( cast(c_birth_year as decimal(12,2))) agg6, avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 from catalog_sales, date_dim, customer_demographics cd1, item, customer, customer_address, customer_demographics cd2 where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and catalog_sales.cs_item_sk = item.i_item_sk and catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk and cd1.cd_gender = 'M' and cd1.cd_education_status = 'College' and customer.c_current_cdemo_sk = cd2.cd_demo_sk and customer.c_current_addr_sk = customer_address.ca_address_sk and c_birth_month in (9,5,12,4,1,10) and d_year = 2001 and ca_state
  in ('ND','WI','AL' ,'NC','OK','MS','TN') group by i_item_id, ca_country, ca_state, ca_county with rollup order by ca_country, ca_state, ca_county, i_item_id limit 100
+PREHOOK: query: explain
+select  i_item_id,
+        ca_country,
+        ca_state, 
+        ca_county,
+        avg( cast(cs_quantity as numeric(12,2))) agg1,
+        avg( cast(cs_list_price as numeric(12,2))) agg2,
+        avg( cast(cs_coupon_amt as numeric(12,2))) agg3,
+        avg( cast(cs_sales_price as numeric(12,2))) agg4,
+        avg( cast(cs_net_profit as numeric(12,2))) agg5,
+        avg( cast(c_birth_year as numeric(12,2))) agg6,
+        avg( cast(cd1.cd_dep_count as numeric(12,2))) agg7
+ from catalog_sales, customer_demographics cd1, 
+      customer_demographics cd2, customer, customer_address, date_dim, item
+ where cs_sold_date_sk = d_date_sk and
+       cs_item_sk = i_item_sk and
+       cs_bill_cdemo_sk = cd1.cd_demo_sk and
+       cs_bill_customer_sk = c_customer_sk and
+       cd1.cd_gender = 'M' and 
+       cd1.cd_education_status = 'College' and
+       c_current_cdemo_sk = cd2.cd_demo_sk and
+       c_current_addr_sk = ca_address_sk and
+       c_birth_month in (9,5,12,4,1,10) and
+       d_year = 2001 and
+       ca_state in ('ND','WI','AL'
+                   ,'NC','OK','MS','TN')
+ group by rollup (i_item_id, ca_country, ca_state, ca_county)
+ order by ca_country,
+        ca_state, 
+        ca_county,
+	i_item_id
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select i_item_id, ca_country, ca_state, ca_county, avg( cast(cs_quantity as decimal(12,2))) agg1, avg( cast(cs_list_price as decimal(12,2))) agg2, avg( cast(cs_coupon_amt as decimal(12,2))) agg3, avg( cast(cs_sales_price as decimal(12,2))) agg4, avg( cast(cs_net_profit as decimal(12,2))) agg5, avg( cast(c_birth_year as decimal(12,2))) agg6, avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 from catalog_sales, date_dim, customer_demographics cd1, item, customer, customer_address, customer_demographics cd2 where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and catalog_sales.cs_item_sk = item.i_item_sk and catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk and cd1.cd_gender = 'M' and cd1.cd_education_status = 'College' and customer.c_current_cdemo_sk = cd2.cd_demo_sk and customer.c_current_addr_sk = customer_address.ca_address_sk and c_birth_month in (9,5,12,4,1,10) and d_year = 2001 and ca_stat
 e in ('ND','WI','AL' ,'NC','OK','MS','TN') group by i_item_id, ca_country, ca_state, ca_county with rollup order by ca_country, ca_state, ca_county, i_item_id limit 100
+POSTHOOK: query: explain
+select  i_item_id,
+        ca_country,
+        ca_state, 
+        ca_county,
+        avg( cast(cs_quantity as numeric(12,2))) agg1,
+        avg( cast(cs_list_price as numeric(12,2))) agg2,
+        avg( cast(cs_coupon_amt as numeric(12,2))) agg3,
+        avg( cast(cs_sales_price as numeric(12,2))) agg4,
+        avg( cast(cs_net_profit as numeric(12,2))) agg5,
+        avg( cast(c_birth_year as numeric(12,2))) agg6,
+        avg( cast(cd1.cd_dep_count as numeric(12,2))) agg7
+ from catalog_sales, customer_demographics cd1, 
+      customer_demographics cd2, customer, customer_address, date_dim, item
+ where cs_sold_date_sk = d_date_sk and
+       cs_item_sk = i_item_sk and
+       cs_bill_cdemo_sk = cd1.cd_demo_sk and
+       cs_bill_customer_sk = c_customer_sk and
+       cd1.cd_gender = 'M' and 
+       cd1.cd_education_status = 'College' and
+       c_current_cdemo_sk = cd2.cd_demo_sk and
+       c_current_addr_sk = ca_address_sk and
+       c_birth_month in (9,5,12,4,1,10) and
+       d_year = 2001 and
+       ca_state in ('ND','WI','AL'
+                   ,'NC','OK','MS','TN')
+ group by rollup (i_item_id, ca_country, ca_state, ca_county)
+ order by ca_country,
+        ca_state, 
+        ca_county,
+	i_item_id
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -89,7 +151,7 @@ Stage-0
                                           Select Operator [SEL_11] (rows=287989836 width=135)
                                             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
                                             Filter Operator [FIL_74] (rows=287989836 width=135)
-                                              predicate:(cs_sold_date_sk is not null and cs_bill_cdemo_sk is not null and cs_item_sk is not null and cs_bill_customer_sk is not null)
+                                              predicate:(cs_bill_cdemo_sk is not null and cs_bill_customer_sk is not null and cs_sold_date_sk is not null and cs_item_sk is not null)
                                               TableScan [TS_9] (rows=287989836 width=135)
                                                 default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_bill_customer_sk","cs_bill_cdemo_sk","cs_item_sk","cs_quantity","cs_list_price","cs_sales_price","cs_coupon_amt","cs_net_profit"]
                         <-Reducer 3 [SIMPLE_EDGE]
@@ -117,7 +179,7 @@ Stage-0
                                     Select Operator [SEL_2] (rows=40000000 width=860)
                                       Output:["_col0","_col1","_col2","_col4"]
                                       Filter Operator [FIL_71] (rows=40000000 width=860)
-                                        predicate:((c_birth_month) IN (9, 5, 12, 4, 1, 10) and c_customer_sk is not null and c_current_addr_sk is not null and c_current_cdemo_sk is not null)
+                                        predicate:((c_birth_month) IN (9, 5, 12, 4, 1, 10) and c_customer_sk is not null and c_current_cdemo_sk is not null and c_current_addr_sk is not null)
                                         TableScan [TS_0] (rows=80000000 width=860)
                                           default@customer,customer,Tbl:COMPLETE,Col:NONE,Output:["c_customer_sk","c_current_cdemo_sk","c_current_addr_sk","c_birth_month","c_birth_year"]
                                 <-Map 7 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query19.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query19.q.out b/ql/src/test/results/clientpositive/perf/query19.q.out
index cebfc3a..fccc0c3 100644
--- a/ql/src/test/results/clientpositive/perf/query19.q.out
+++ b/ql/src/test/results/clientpositive/perf/query19.q.out
@@ -1,6 +1,50 @@
-PREHOOK: query: explain select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, sum(ss_ext_sales_price) ext_price from date_dim, store_sales, item,customer,customer_address,store where date_dim.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and i_manager_id=7 and d_moy=11 and d_year=1999 and store_sales.ss_customer_sk = customer.c_customer_sk and customer.c_current_addr_sk = customer_address.ca_address_sk and substr(ca_zip,1,5) <> substr(s_zip,1,5) and store_sales.ss_store_sk = store.s_store_sk group by i_brand ,i_brand_id ,i_manufact_id ,i_manufact order by ext_price desc ,i_brand ,i_brand_id ,i_manufact_id ,i_manufact limit 100
+PREHOOK: query: explain
+select  i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact,
+ 	sum(ss_ext_sales_price) ext_price
+ from date_dim, store_sales, item,customer,customer_address,store
+ where d_date_sk = ss_sold_date_sk
+   and ss_item_sk = i_item_sk
+   and i_manager_id=7
+   and d_moy=11
+   and d_year=1999
+   and ss_customer_sk = c_customer_sk 
+   and c_current_addr_sk = ca_address_sk
+   and substr(ca_zip,1,5) <> substr(s_zip,1,5) 
+   and ss_store_sk = s_store_sk 
+ group by i_brand
+      ,i_brand_id
+      ,i_manufact_id
+      ,i_manufact
+ order by ext_price desc
+         ,i_brand
+         ,i_brand_id
+         ,i_manufact_id
+         ,i_manufact
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, sum(ss_ext_sales_price) ext_price from date_dim, store_sales, item,customer,customer_address,store where date_dim.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and i_manager_id=7 and d_moy=11 and d_year=1999 and store_sales.ss_customer_sk = customer.c_customer_sk and customer.c_current_addr_sk = customer_address.ca_address_sk and substr(ca_zip,1,5) <> substr(s_zip,1,5) and store_sales.ss_store_sk = store.s_store_sk group by i_brand ,i_brand_id ,i_manufact_id ,i_manufact order by ext_price desc ,i_brand ,i_brand_id ,i_manufact_id ,i_manufact limit 100
+POSTHOOK: query: explain
+select  i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact,
+ 	sum(ss_ext_sales_price) ext_price
+ from date_dim, store_sales, item,customer,customer_address,store
+ where d_date_sk = ss_sold_date_sk
+   and ss_item_sk = i_item_sk
+   and i_manager_id=7
+   and d_moy=11
+   and d_year=1999
+   and ss_customer_sk = c_customer_sk 
+   and c_current_addr_sk = ca_address_sk
+   and substr(ca_zip,1,5) <> substr(s_zip,1,5) 
+   and ss_store_sk = s_store_sk 
+ group by i_brand
+      ,i_brand_id
+      ,i_manufact_id
+      ,i_manufact
+ order by ext_price desc
+         ,i_brand
+         ,i_brand_id
+         ,i_manufact_id
+         ,i_manufact
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query2.q.out b/ql/src/test/results/clientpositive/perf/query2.q.out
new file mode 100644
index 0000000..50d7f7b
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query2.q.out
@@ -0,0 +1,259 @@
+PREHOOK: query: explain
+with wscs as
+ (select sold_date_sk
+        ,sales_price
+  from (select ws_sold_date_sk sold_date_sk
+              ,ws_ext_sales_price sales_price
+        from web_sales) x
+        union all
+       (select cs_sold_date_sk sold_date_sk
+              ,cs_ext_sales_price sales_price
+        from catalog_sales)),
+ wswscs as 
+ (select d_week_seq,
+        sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales
+ from wscs
+     ,date_dim
+ where d_date_sk = sold_date_sk
+ group by d_week_seq)
+ select d_week_seq1
+       ,round(sun_sales1/sun_sales2,2)
+       ,round(mon_sales1/mon_sales2,2)
+       ,round(tue_sales1/tue_sales2,2)
+       ,round(wed_sales1/wed_sales2,2)
+       ,round(thu_sales1/thu_sales2,2)
+       ,round(fri_sales1/fri_sales2,2)
+       ,round(sat_sales1/sat_sales2,2)
+ from
+ (select wswscs.d_week_seq d_week_seq1
+        ,sun_sales sun_sales1
+        ,mon_sales mon_sales1
+        ,tue_sales tue_sales1
+        ,wed_sales wed_sales1
+        ,thu_sales thu_sales1
+        ,fri_sales fri_sales1
+        ,sat_sales sat_sales1
+  from wswscs,date_dim 
+  where date_dim.d_week_seq = wswscs.d_week_seq and
+        d_year = 2001) y,
+ (select wswscs.d_week_seq d_week_seq2
+        ,sun_sales sun_sales2
+        ,mon_sales mon_sales2
+        ,tue_sales tue_sales2
+        ,wed_sales wed_sales2
+        ,thu_sales thu_sales2
+        ,fri_sales fri_sales2
+        ,sat_sales sat_sales2
+  from wswscs
+      ,date_dim 
+  where date_dim.d_week_seq = wswscs.d_week_seq and
+        d_year = 2001+1) z
+ where d_week_seq1=d_week_seq2-53
+ order by d_week_seq1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+with wscs as
+ (select sold_date_sk
+        ,sales_price
+  from (select ws_sold_date_sk sold_date_sk
+              ,ws_ext_sales_price sales_price
+        from web_sales) x
+        union all
+       (select cs_sold_date_sk sold_date_sk
+              ,cs_ext_sales_price sales_price
+        from catalog_sales)),
+ wswscs as 
+ (select d_week_seq,
+        sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales
+ from wscs
+     ,date_dim
+ where d_date_sk = sold_date_sk
+ group by d_week_seq)
+ select d_week_seq1
+       ,round(sun_sales1/sun_sales2,2)
+       ,round(mon_sales1/mon_sales2,2)
+       ,round(tue_sales1/tue_sales2,2)
+       ,round(wed_sales1/wed_sales2,2)
+       ,round(thu_sales1/thu_sales2,2)
+       ,round(fri_sales1/fri_sales2,2)
+       ,round(sat_sales1/sat_sales2,2)
+ from
+ (select wswscs.d_week_seq d_week_seq1
+        ,sun_sales sun_sales1
+        ,mon_sales mon_sales1
+        ,tue_sales tue_sales1
+        ,wed_sales wed_sales1
+        ,thu_sales thu_sales1
+        ,fri_sales fri_sales1
+        ,sat_sales sat_sales1
+  from wswscs,date_dim 
+  where date_dim.d_week_seq = wswscs.d_week_seq and
+        d_year = 2001) y,
+ (select wswscs.d_week_seq d_week_seq2
+        ,sun_sales sun_sales2
+        ,mon_sales mon_sales2
+        ,tue_sales tue_sales2
+        ,wed_sales wed_sales2
+        ,thu_sales thu_sales2
+        ,fri_sales fri_sales2
+        ,sat_sales sat_sales2
+  from wswscs
+      ,date_dim 
+  where date_dim.d_week_seq = wswscs.d_week_seq and
+        d_year = 2001+1) z
+ where d_week_seq1=d_week_seq2-53
+ order by d_week_seq1
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Map 1 <- Union 2 (CONTAINS)
+Map 14 <- Union 15 (CONTAINS)
+Map 16 <- Union 15 (CONTAINS)
+Map 8 <- Union 2 (CONTAINS)
+Reducer 10 <- Map 9 (SIMPLE_EDGE), Union 15 (SIMPLE_EDGE)
+Reducer 11 <- Reducer 10 (SIMPLE_EDGE)
+Reducer 12 <- Map 13 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE)
+Reducer 3 <- Map 9 (SIMPLE_EDGE), Union 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Map 13 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 12 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 7
+      File Output Operator [FS_60]
+        Select Operator [SEL_59] (rows=287491028 width=135)
+          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+        <-Reducer 6 [SIMPLE_EDGE]
+          SHUFFLE [RS_58]
+            Select Operator [SEL_57] (rows=287491028 width=135)
+              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+              Merge Join Operator [MERGEJOIN_95] (rows=287491028 width=135)
+                Conds:RS_54._col0=RS_55.(_col0 - 53)(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col9","_col10","_col11","_col12","_col13","_col14","_col15"]
+              <-Reducer 12 [SIMPLE_EDGE]
+                SHUFFLE [RS_55]
+                  PartitionCols:(_col0 - 53)
+                  Merge Join Operator [MERGEJOIN_94] (rows=261355475 width=135)
+                    Conds:RS_50._col0=RS_51._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+                  <-Map 13 [SIMPLE_EDGE]
+                    SHUFFLE [RS_51]
+                      PartitionCols:_col0
+                      Select Operator [SEL_49] (rows=36524 width=1119)
+                        Output:["_col0"]
+                        Filter Operator [FIL_90] (rows=36524 width=1119)
+                          predicate:((d_year = 2002) and d_week_seq is not null)
+                          TableScan [TS_20] (rows=73049 width=1119)
+                            default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_week_seq","d_year"]
+                  <-Reducer 11 [SIMPLE_EDGE]
+                    SHUFFLE [RS_50]
+                      PartitionCols:_col0
+                      Group By Operator [GBY_45] (rows=237595882 width=135)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)"],keys:KEY._col0
+                      <-Reducer 10 [SIMPLE_EDGE]
+                        SHUFFLE [RS_44]
+                          PartitionCols:_col0
+                          Group By Operator [GBY_43] (rows=475191764 width=135)
+                            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col1)","sum(_col2)","sum(_col3)","sum(_col4)","sum(_col5)","sum(_col6)","sum(_col7)"],keys:_col0
+                            Select Operator [SEL_41] (rows=475191764 width=135)
+                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+                              Merge Join Operator [MERGEJOIN_93] (rows=475191764 width=135)
+                                Conds:Union 15._col0=RS_39._col0(Inner),Output:["_col1","_col3","_col4"]
+                              <-Map 9 [SIMPLE_EDGE]
+                                SHUFFLE [RS_39]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_37] (rows=73049 width=1119)
+                                    Output:["_col0","_col1","_col2"]
+                                    Filter Operator [FIL_89] (rows=73049 width=1119)
+                                      predicate:(d_date_sk is not null and d_week_seq is not null)
+                                      TableScan [TS_8] (rows=73049 width=1119)
+                                        default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_week_seq","d_day_name"]
+                              <-Union 15 [SIMPLE_EDGE]
+                                <-Map 14 [CONTAINS]
+                                  Reduce Output Operator [RS_38]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_29] (rows=144002668 width=135)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_87] (rows=144002668 width=135)
+                                        predicate:ws_sold_date_sk is not null
+                                        TableScan [TS_27] (rows=144002668 width=135)
+                                          Output:["ws_sold_date_sk","ws_ext_sales_price"]
+                                <-Map 16 [CONTAINS]
+                                  Reduce Output Operator [RS_38]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_32] (rows=287989836 width=135)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_88] (rows=287989836 width=135)
+                                        predicate:cs_sold_date_sk is not null
+                                        TableScan [TS_30] (rows=287989836 width=135)
+                                          Output:["cs_sold_date_sk","cs_ext_sales_price"]
+              <-Reducer 5 [SIMPLE_EDGE]
+                SHUFFLE [RS_54]
+                  PartitionCols:_col0
+                  Merge Join Operator [MERGEJOIN_92] (rows=261355475 width=135)
+                    Conds:RS_23._col0=RS_24._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+                  <-Map 13 [SIMPLE_EDGE]
+                    SHUFFLE [RS_24]
+                      PartitionCols:_col0
+                      Select Operator [SEL_22] (rows=36524 width=1119)
+                        Output:["_col0"]
+                        Filter Operator [FIL_86] (rows=36524 width=1119)
+                          predicate:((d_year = 2001) and d_week_seq is not null)
+                           Please refer to the previous TableScan [TS_20]
+                  <-Reducer 4 [SIMPLE_EDGE]
+                    SHUFFLE [RS_23]
+                      PartitionCols:_col0
+                      Group By Operator [GBY_18] (rows=237595882 width=135)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)"],keys:KEY._col0
+                      <-Reducer 3 [SIMPLE_EDGE]
+                        SHUFFLE [RS_17]
+                          PartitionCols:_col0
+                          Group By Operator [GBY_16] (rows=475191764 width=135)
+                            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col1)","sum(_col2)","sum(_col3)","sum(_col4)","sum(_col5)","sum(_col6)","sum(_col7)"],keys:_col0
+                            Select Operator [SEL_14] (rows=475191764 width=135)
+                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+                              Merge Join Operator [MERGEJOIN_91] (rows=475191764 width=135)
+                                Conds:Union 2._col0=RS_12._col0(Inner),Output:["_col1","_col3","_col4"]
+                              <-Map 9 [SIMPLE_EDGE]
+                                SHUFFLE [RS_12]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_10] (rows=73049 width=1119)
+                                    Output:["_col0","_col1","_col2"]
+                                    Filter Operator [FIL_85] (rows=73049 width=1119)
+                                      predicate:(d_date_sk is not null and d_week_seq is not null)
+                                       Please refer to the previous TableScan [TS_8]
+                              <-Union 2 [SIMPLE_EDGE]
+                                <-Map 1 [CONTAINS]
+                                  Reduce Output Operator [RS_11]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_2] (rows=144002668 width=135)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_83] (rows=144002668 width=135)
+                                        predicate:ws_sold_date_sk is not null
+                                        TableScan [TS_0] (rows=144002668 width=135)
+                                          Output:["ws_sold_date_sk","ws_ext_sales_price"]
+                                <-Map 8 [CONTAINS]
+                                  Reduce Output Operator [RS_11]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_5] (rows=287989836 width=135)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_84] (rows=287989836 width=135)
+                                        predicate:cs_sold_date_sk is not null
+                                        TableScan [TS_3] (rows=287989836 width=135)
+                                          Output:["cs_sold_date_sk","cs_ext_sales_price"]
+

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query20.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query20.q.out b/ql/src/test/results/clientpositive/perf/query20.q.out
index cff652f..c07141b 100644
--- a/ql/src/test/results/clientpositive/perf/query20.q.out
+++ b/ql/src/test/results/clientpositive/perf/query20.q.out
@@ -1,6 +1,58 @@
-PREHOOK: query: explain select i_item_desc ,i_category ,i_class ,i_current_price ,i_item_id ,sum(cs_ext_sales_price) as itemrevenue ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over (partition by i_class) as revenueratio from catalog_sales ,item ,date_dim where catalog_sales.cs_item_sk = item.i_item_sk and i_category in ('Jewelry', 'Sports', 'Books') and catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and d_date between '2001-01-12' and '2001-02-11' group by i_item_id ,i_item_desc ,i_category ,i_class ,i_current_price order by i_category ,i_class ,i_item_id ,i_item_desc ,revenueratio limit 100
+PREHOOK: query: explain
+select  i_item_desc 
+       ,i_category 
+       ,i_class 
+       ,i_current_price
+       ,sum(cs_ext_sales_price) as itemrevenue 
+       ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over
+           (partition by i_class) as revenueratio
+ from	catalog_sales
+     ,item 
+     ,date_dim
+ where cs_item_sk = i_item_sk 
+   and i_category in ('Jewelry', 'Sports', 'Books')
+   and cs_sold_date_sk = d_date_sk
+ and d_date between cast('2001-01-12' as date) 
+ 				and (cast('2001-01-12' as date) + 30 days)
+ group by i_item_id
+         ,i_item_desc 
+         ,i_category
+         ,i_class
+         ,i_current_price
+ order by i_category
+         ,i_class
+         ,i_item_id
+         ,i_item_desc
+         ,revenueratio
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select i_item_desc ,i_category ,i_class ,i_current_price ,i_item_id ,sum(cs_ext_sales_price) as itemrevenue ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over (partition by i_class) as revenueratio from catalog_sales ,item ,date_dim where catalog_sales.cs_item_sk = item.i_item_sk and i_category in ('Jewelry', 'Sports', 'Books') and catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and d_date between '2001-01-12' and '2001-02-11' group by i_item_id ,i_item_desc ,i_category ,i_class ,i_current_price order by i_category ,i_class ,i_item_id ,i_item_desc ,revenueratio limit 100
+POSTHOOK: query: explain
+select  i_item_desc 
+       ,i_category 
+       ,i_class 
+       ,i_current_price
+       ,sum(cs_ext_sales_price) as itemrevenue 
+       ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over
+           (partition by i_class) as revenueratio
+ from	catalog_sales
+     ,item 
+     ,date_dim
+ where cs_item_sk = i_item_sk 
+   and i_category in ('Jewelry', 'Sports', 'Books')
+   and cs_sold_date_sk = d_date_sk
+ and d_date between cast('2001-01-12' as date) 
+ 				and (cast('2001-01-12' as date) + 30 days)
+ group by i_item_id
+         ,i_item_desc 
+         ,i_category
+         ,i_class
+         ,i_current_price
+ order by i_category
+         ,i_class
+         ,i_item_id
+         ,i_item_desc
+         ,revenueratio
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -13,14 +65,14 @@ Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
       Reducer 6
-      File Output Operator [FS_28]
+      File Output Operator [FS_29]
         Limit [LIM_27] (rows=100 width=135)
           Number of rows:100
           Select Operator [SEL_26] (rows=174233858 width=135)
-            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+            Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           <-Reducer 5 [SIMPLE_EDGE]
             SHUFFLE [RS_25]
               Select Operator [SEL_23] (rows=174233858 width=135)
@@ -41,28 +93,28 @@ Stage-0
                             PartitionCols:_col0, _col1, _col2, _col3, _col4
                             Group By Operator [GBY_16] (rows=348467716 width=135)
                               Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col2)"],keys:_col10, _col9, _col6, _col7, _col8
-                              Merge Join Operator [MERGEJOIN_38] (rows=348467716 width=135)
+                              Merge Join Operator [MERGEJOIN_39] (rows=348467716 width=135)
                                 Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col2","_col6","_col7","_col8","_col9","_col10"]
                               <-Map 8 [SIMPLE_EDGE]
                                 SHUFFLE [RS_13]
                                   PartitionCols:_col0
                                   Select Operator [SEL_8] (rows=231000 width=1436)
                                     Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                    Filter Operator [FIL_36] (rows=231000 width=1436)
+                                    Filter Operator [FIL_37] (rows=231000 width=1436)
                                       predicate:((i_category) IN ('Jewelry', 'Sports', 'Books') and i_item_sk is not null)
                                       TableScan [TS_6] (rows=462000 width=1436)
                                         default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_item_id","i_item_desc","i_current_price","i_class","i_category"]
                               <-Reducer 2 [SIMPLE_EDGE]
                                 SHUFFLE [RS_12]
                                   PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_37] (rows=316788826 width=135)
+                                  Merge Join Operator [MERGEJOIN_38] (rows=316788826 width=135)
                                     Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col1","_col2"]
                                   <-Map 1 [SIMPLE_EDGE]
                                     SHUFFLE [RS_9]
                                       PartitionCols:_col0
                                       Select Operator [SEL_2] (rows=287989836 width=135)
                                         Output:["_col0","_col1","_col2"]
-                                        Filter Operator [FIL_34] (rows=287989836 width=135)
+                                        Filter Operator [FIL_35] (rows=287989836 width=135)
                                           predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
                                           TableScan [TS_0] (rows=287989836 width=135)
                                             default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_ext_sales_price"]
@@ -71,8 +123,8 @@ Stage-0
                                       PartitionCols:_col0
                                       Select Operator [SEL_5] (rows=8116 width=1119)
                                         Output:["_col0"]
-                                        Filter Operator [FIL_35] (rows=8116 width=1119)
-                                          predicate:(d_date BETWEEN '2001-01-12' AND '2001-02-11' and d_date_sk is not null)
+                                        Filter Operator [FIL_36] (rows=8116 width=1119)
+                                          predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 2001-01-12 00:00:00.0 AND 2001-02-11 00:00:00.0 and d_date_sk is not null)
                                           TableScan [TS_3] (rows=73049 width=1119)
                                             default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query23.q.out b/ql/src/test/results/clientpositive/perf/query23.q.out
index 5794301..b8cdad3 100644
--- a/ql/src/test/results/clientpositive/perf/query23.q.out
+++ b/ql/src/test/results/clientpositive/perf/query23.q.out
@@ -1,6 +1,7 @@
-Warning: Shuffle Join MERGEJOIN[369][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 30' is a cross product
 Warning: Shuffle Join MERGEJOIN[367][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 25' is a cross product
-PREHOOK: query: explain with frequent_ss_items as 
+Warning: Shuffle Join MERGEJOIN[369][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 30' is a cross product
+PREHOOK: query: explain
+with frequent_ss_items as 
  (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
   from store_sales
       ,date_dim 
@@ -50,7 +51,8 @@ from
          and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer))) y
  limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with frequent_ss_items as 
+POSTHOOK: query: explain
+with frequent_ss_items as 
  (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
   from store_sales
       ,date_dim 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query24.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query24.q.out b/ql/src/test/results/clientpositive/perf/query24.q.out
index 105871f..2aa0c19 100644
--- a/ql/src/test/results/clientpositive/perf/query24.q.out
+++ b/ql/src/test/results/clientpositive/perf/query24.q.out
@@ -1,99 +1,101 @@
 Warning: Shuffle Join MERGEJOIN[154][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 8' is a cross product
-PREHOOK: query: explain with ssales as
-        (select c_last_name
-              ,c_first_name
-              ,s_store_name
-              ,ca_state
-              ,s_state
-              ,i_color
-              ,i_current_price
-              ,i_manager_id
-              ,i_units
-              ,i_size
-              ,sum(ss_sales_price) netpaid
-        from store_sales
-            ,store_returns
-            ,store
-            ,item
-            ,customer
-            ,customer_address
-        where ss_ticket_number = sr_ticket_number
-          and ss_item_sk = sr_item_sk
-          and ss_customer_sk = c_customer_sk
-          and ss_item_sk = i_item_sk
-          and ss_store_sk = s_store_sk
-          and c_birth_country = upper(ca_country)
-          and s_zip = ca_zip
-        and s_market_id=7
-        group by c_last_name
-                ,c_first_name
-                ,s_store_name
-                ,ca_state
-                ,s_state
-                ,i_color
-                ,i_current_price
-                ,i_manager_id
-                ,i_units
-                ,i_size)
-        select c_last_name
-              ,c_first_name
-              ,s_store_name
-              ,sum(netpaid) paid
-        from ssales
-        where i_color = 'orchid'
-        group by c_last_name
-                ,c_first_name
-                ,s_store_name
-        having sum(netpaid) > (select 0.05*avg(netpaid)
-                                         from ssales)
+PREHOOK: query: explain
+with ssales as
+(select c_last_name
+      ,c_first_name
+      ,s_store_name
+      ,ca_state
+      ,s_state
+      ,i_color
+      ,i_current_price
+      ,i_manager_id
+      ,i_units
+      ,i_size
+      ,sum(ss_sales_price) netpaid
+from store_sales
+    ,store_returns
+    ,store
+    ,item
+    ,customer
+    ,customer_address
+where ss_ticket_number = sr_ticket_number
+  and ss_item_sk = sr_item_sk
+  and ss_customer_sk = c_customer_sk
+  and ss_item_sk = i_item_sk
+  and ss_store_sk = s_store_sk
+  and c_birth_country = upper(ca_country)
+  and s_zip = ca_zip
+and s_market_id=7
+group by c_last_name
+        ,c_first_name
+        ,s_store_name
+        ,ca_state
+        ,s_state
+        ,i_color
+        ,i_current_price
+        ,i_manager_id
+        ,i_units
+        ,i_size)
+select c_last_name
+      ,c_first_name
+      ,s_store_name
+      ,sum(netpaid) paid
+from ssales
+where i_color = 'orchid'
+group by c_last_name
+        ,c_first_name
+        ,s_store_name
+having sum(netpaid) > (select 0.05*avg(netpaid)
+                                 from ssales)
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with ssales as
-        (select c_last_name
-              ,c_first_name
-              ,s_store_name
-              ,ca_state
-              ,s_state
-              ,i_color
-              ,i_current_price
-              ,i_manager_id
-              ,i_units
-              ,i_size
-              ,sum(ss_sales_price) netpaid
-        from store_sales
-            ,store_returns
-            ,store
-            ,item
-            ,customer
-            ,customer_address
-        where ss_ticket_number = sr_ticket_number
-          and ss_item_sk = sr_item_sk
-          and ss_customer_sk = c_customer_sk
-          and ss_item_sk = i_item_sk
-          and ss_store_sk = s_store_sk
-          and c_birth_country = upper(ca_country)
-          and s_zip = ca_zip
-        and s_market_id=7
-        group by c_last_name
-                ,c_first_name
-                ,s_store_name
-                ,ca_state
-                ,s_state
-                ,i_color
-                ,i_current_price
-                ,i_manager_id
-                ,i_units
-                ,i_size)
-        select c_last_name
-              ,c_first_name
-              ,s_store_name
-              ,sum(netpaid) paid
-        from ssales
-        where i_color = 'orchid'
-        group by c_last_name
-                ,c_first_name
-                ,s_store_name
-        having sum(netpaid) > (select 0.05*avg(netpaid)
-                                         from ssales)
+POSTHOOK: query: explain
+with ssales as
+(select c_last_name
+      ,c_first_name
+      ,s_store_name
+      ,ca_state
+      ,s_state
+      ,i_color
+      ,i_current_price
+      ,i_manager_id
+      ,i_units
+      ,i_size
+      ,sum(ss_sales_price) netpaid
+from store_sales
+    ,store_returns
+    ,store
+    ,item
+    ,customer
+    ,customer_address
+where ss_ticket_number = sr_ticket_number
+  and ss_item_sk = sr_item_sk
+  and ss_customer_sk = c_customer_sk
+  and ss_item_sk = i_item_sk
+  and ss_store_sk = s_store_sk
+  and c_birth_country = upper(ca_country)
+  and s_zip = ca_zip
+and s_market_id=7
+group by c_last_name
+        ,c_first_name
+        ,s_store_name
+        ,ca_state
+        ,s_state
+        ,i_color
+        ,i_current_price
+        ,i_manager_id
+        ,i_units
+        ,i_size)
+select c_last_name
+      ,c_first_name
+      ,s_store_name
+      ,sum(netpaid) paid
+from ssales
+where i_color = 'orchid'
+group by c_last_name
+        ,c_first_name
+        ,s_store_name
+having sum(netpaid) > (select 0.05*avg(netpaid)
+                                 from ssales)
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 


[07/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query62.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query62.q.out b/ql/src/test/results/clientpositive/perf/query62.q.out
new file mode 100644
index 0000000..ae50787
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query62.q.out
@@ -0,0 +1,164 @@
+PREHOOK: query: explain
+select  
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,web_name
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end)  as `30 days` 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and 
+                 (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end )  as `31-60 days` 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and 
+                 (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end)  as `61-90 days` 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and
+                 (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end)  as `91-120 days` 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk  > 120) then 1 else 0 end)  as `>120 days` 
+from
+   web_sales
+  ,warehouse
+  ,ship_mode
+  ,web_site
+  ,date_dim
+where
+    d_month_seq between 1212 and 1212 + 11
+and ws_ship_date_sk   = d_date_sk
+and ws_warehouse_sk   = w_warehouse_sk
+and ws_ship_mode_sk   = sm_ship_mode_sk
+and ws_web_site_sk    = web_site_sk
+group by
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,web_name
+order by substr(w_warehouse_name,1,20)
+        ,sm_type
+       ,web_name
+limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select  
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,web_name
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end)  as `30 days` 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and 
+                 (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end )  as `31-60 days` 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and 
+                 (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end)  as `61-90 days` 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and
+                 (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end)  as `91-120 days` 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk  > 120) then 1 else 0 end)  as `>120 days` 
+from
+   web_sales
+  ,warehouse
+  ,ship_mode
+  ,web_site
+  ,date_dim
+where
+    d_month_seq between 1212 and 1212 + 11
+and ws_ship_date_sk   = d_date_sk
+and ws_warehouse_sk   = w_warehouse_sk
+and ws_ship_mode_sk   = sm_ship_mode_sk
+and ws_web_site_sk    = web_site_sk
+group by
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,web_name
+order by substr(w_warehouse_name,1,20)
+        ,sm_type
+       ,web_name
+limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 3 <- Map 9 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 10 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Map 11 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 7
+      File Output Operator [FS_37]
+        Limit [LIM_35] (rows=100 width=135)
+          Number of rows:100
+          Select Operator [SEL_34] (rows=105417161 width=135)
+            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+          <-Reducer 6 [SIMPLE_EDGE]
+            SHUFFLE [RS_33]
+              Select Operator [SEL_32] (rows=105417161 width=135)
+                Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
+                Group By Operator [GBY_31] (rows=105417161 width=135)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)"],keys:KEY._col0, KEY._col1, KEY._col2
+                <-Reducer 5 [SIMPLE_EDGE]
+                  SHUFFLE [RS_30]
+                    PartitionCols:_col0, _col1, _col2
+                    Group By Operator [GBY_29] (rows=210834322 width=135)
+                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col3)","sum(_col4)","sum(_col5)","sum(_col6)","sum(_col7)"],keys:_col0, _col1, _col2
+                      Select Operator [SEL_27] (rows=210834322 width=135)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+                        Merge Join Operator [MERGEJOIN_60] (rows=210834322 width=135)
+                          Conds:RS_24._col3=RS_25._col0(Inner),Output:["_col0","_col1","_col8","_col10","_col12"]
+                        <-Map 11 [SIMPLE_EDGE]
+                          SHUFFLE [RS_25]
+                            PartitionCols:_col0
+                            Select Operator [SEL_14] (rows=1 width=0)
+                              Output:["_col0","_col1"]
+                              Filter Operator [FIL_56] (rows=1 width=0)
+                                predicate: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_type"]
+                        <-Reducer 4 [SIMPLE_EDGE]
+                          SHUFFLE [RS_24]
+                            PartitionCols:_col3
+                            Merge Join Operator [MERGEJOIN_59] (rows=191667562 width=135)
+                              Conds:RS_21._col4=RS_22._col0(Inner),Output:["_col0","_col1","_col3","_col8","_col10"]
+                            <-Map 10 [SIMPLE_EDGE]
+                              SHUFFLE [RS_22]
+                                PartitionCols:_col0
+                                Select Operator [SEL_11] (rows=27 width=1029)
+                                  Output:["_col0","_col1"]
+                                  Filter Operator [FIL_55] (rows=27 width=1029)
+                                    predicate:w_warehouse_sk is not null
+                                    TableScan [TS_9] (rows=27 width=1029)
+                                      default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name"]
+                            <-Reducer 3 [SIMPLE_EDGE]
+                              SHUFFLE [RS_21]
+                                PartitionCols:_col4
+                                Merge Join Operator [MERGEJOIN_58] (rows=174243235 width=135)
+                                  Conds:RS_18._col2=RS_19._col0(Inner),Output:["_col0","_col1","_col3","_col4","_col8"]
+                                <-Map 9 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_19]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_8] (rows=84 width=1850)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_54] (rows=84 width=1850)
+                                        predicate:web_site_sk is not null
+                                        TableScan [TS_6] (rows=84 width=1850)
+                                          default@web_site,web_site,Tbl:COMPLETE,Col:NONE,Output:["web_site_sk","web_name"]
+                                <-Reducer 2 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_18]
+                                    PartitionCols:_col2
+                                    Merge Join Operator [MERGEJOIN_57] (rows=158402938 width=135)
+                                      Conds:RS_15._col1=RS_16._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4"]
+                                    <-Map 1 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_15]
+                                        PartitionCols:_col1
+                                        Select Operator [SEL_2] (rows=144002668 width=135)
+                                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                                          Filter Operator [FIL_52] (rows=144002668 width=135)
+                                            predicate:(ws_warehouse_sk is not null and ws_ship_mode_sk is not null and ws_web_site_sk is not null and ws_ship_date_sk is not null)
+                                            TableScan [TS_0] (rows=144002668 width=135)
+                                              default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_ship_date_sk","ws_web_site_sk","ws_ship_mode_sk","ws_warehouse_sk"]
+                                    <-Map 8 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_16]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_5] (rows=8116 width=1119)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_53] (rows=8116 width=1119)
+                                            predicate:(d_month_seq BETWEEN 1212 AND 1223 and d_date_sk is not null)
+                                            TableScan [TS_3] (rows=73049 width=1119)
+                                              default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_month_seq"]
+

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query63.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query63.q.out b/ql/src/test/results/clientpositive/perf/query63.q.out
new file mode 100644
index 0000000..1f0f184
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query63.q.out
@@ -0,0 +1,143 @@
+PREHOOK: query: explain
+select  * 
+from (select i_manager_id
+             ,sum(ss_sales_price) sum_sales
+             ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales
+      from item
+          ,store_sales
+          ,date_dim
+          ,store
+      where ss_item_sk = i_item_sk
+        and ss_sold_date_sk = d_date_sk
+        and ss_store_sk = s_store_sk
+        and d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11)
+        and ((    i_category in ('Books','Children','Electronics')
+              and i_class in ('personal','portable','refernece','self-help')
+              and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+		                  'exportiunivamalg #9','scholaramalgamalg #9'))
+           or(    i_category in ('Women','Music','Men')
+              and i_class in ('accessories','classical','fragrances','pants')
+              and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+		                 'importoamalg #1')))
+group by i_manager_id, d_moy) tmp1
+where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+order by i_manager_id
+        ,avg_monthly_sales
+        ,sum_sales
+limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select  * 
+from (select i_manager_id
+             ,sum(ss_sales_price) sum_sales
+             ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales
+      from item
+          ,store_sales
+          ,date_dim
+          ,store
+      where ss_item_sk = i_item_sk
+        and ss_sold_date_sk = d_date_sk
+        and ss_store_sk = s_store_sk
+        and d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11)
+        and ((    i_category in ('Books','Children','Electronics')
+              and i_class in ('personal','portable','refernece','self-help')
+              and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+		                  'exportiunivamalg #9','scholaramalgamalg #9'))
+           or(    i_category in ('Women','Music','Men')
+              and i_class in ('accessories','classical','fragrances','pants')
+              and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+		                 'importoamalg #1')))
+group by i_manager_id, d_moy) tmp1
+where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+order by i_manager_id
+        ,avg_monthly_sales
+        ,sum_sales
+limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 6
+      File Output Operator [FS_36]
+        Limit [LIM_35] (rows=100 width=88)
+          Number of rows:100
+          Select Operator [SEL_34] (rows=191662559 width=88)
+            Output:["_col0","_col1","_col2"]
+          <-Reducer 5 [SIMPLE_EDGE]
+            SHUFFLE [RS_33]
+              Select Operator [SEL_30] (rows=191662559 width=88)
+                Output:["_col0","_col1","_col2"]
+                Filter Operator [FIL_46] (rows=191662559 width=88)
+                  predicate:CASE WHEN ((avg_window_0 > 0)) THEN (((abs((_col2 - avg_window_0)) / avg_window_0) > 0.1)) ELSE (null) END
+                  Select Operator [SEL_29] (rows=383325119 width=88)
+                    Output:["avg_window_0","_col0","_col2"]
+                    PTF Operator [PTF_28] (rows=383325119 width=88)
+                      Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col0 ASC NULLS FIRST","partition by:":"_col0"}]
+                      Select Operator [SEL_25] (rows=383325119 width=88)
+                        Output:["_col0","_col2"]
+                        Group By Operator [GBY_24] (rows=383325119 width=88)
+                          Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
+                        <-Reducer 4 [SIMPLE_EDGE]
+                          SHUFFLE [RS_23]
+                            PartitionCols:_col0
+                            Group By Operator [GBY_22] (rows=766650239 width=88)
+                              Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col8, _col11
+                              Merge Join Operator [MERGEJOIN_54] (rows=766650239 width=88)
+                                Conds:RS_18._col2=RS_19._col0(Inner),Output:["_col3","_col8","_col11"]
+                              <-Map 9 [SIMPLE_EDGE]
+                                SHUFFLE [RS_19]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_11] (rows=1704 width=1910)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_50] (rows=1704 width=1910)
+                                      predicate:s_store_sk is not null
+                                      TableScan [TS_9] (rows=1704 width=1910)
+                                        default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk"]
+                              <-Reducer 3 [SIMPLE_EDGE]
+                                SHUFFLE [RS_18]
+                                  PartitionCols:_col2
+                                  Merge Join Operator [MERGEJOIN_53] (rows=696954748 width=88)
+                                    Conds:RS_15._col0=RS_16._col0(Inner),Output:["_col2","_col3","_col8","_col11"]
+                                  <-Map 8 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_16]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_8] (rows=36525 width=1119)
+                                        Output:["_col0","_col2"]
+                                        Filter Operator [FIL_49] (rows=36525 width=1119)
+                                          predicate:((d_month_seq) IN (1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222, 1223) 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_month_seq","d_moy"]
+                                  <-Reducer 2 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_15]
+                                      PartitionCols:_col0
+                                      Merge Join Operator [MERGEJOIN_52] (rows=633595212 width=88)
+                                        Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col0","_col2","_col3","_col8"]
+                                      <-Map 1 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_12]
+                                          PartitionCols:_col1
+                                          Select Operator [SEL_2] (rows=575995635 width=88)
+                                            Output:["_col0","_col1","_col2","_col3"]
+                                            Filter Operator [FIL_47] (rows=575995635 width=88)
+                                              predicate:(ss_item_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null)
+                                              TableScan [TS_0] (rows=575995635 width=88)
+                                                default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_store_sk","ss_sales_price"]
+                                      <-Map 7 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_13]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_5] (rows=115500 width=1436)
+                                            Output:["_col0","_col4"]
+                                            Filter Operator [FIL_48] (rows=115500 width=1436)
+                                              predicate:(((i_class) IN ('personal', 'portable', 'refernece', 'self-help') or (i_class) IN ('accessories', 'classical', 'fragrances', 'pants')) and ((i_brand) IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9') or (i_brand) IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')) and ((i_category) IN ('Books', 'Children', 'Electronics') or (i_category) IN ('Women', 'Music', 'Men')) and (((i_category) IN ('Books', 'Children', 'Electronics') and (i_class) IN ('personal', 'portable', 'refernece', 'self-help') and (i_brand) IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) or ((i_category) IN ('Women', 'Music', 'Men') and (i_class) IN ('accessories', 'classical', 'fragrances', 'pants') and (i_brand) IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1'))) and i_item_sk is not null)
+                                              TableScan [TS_3] (rows=462000 width=1436)
+                                                default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand","i_class","i_category","i_manager_id"]
+

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query64.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query64.q.out b/ql/src/test/results/clientpositive/perf/query64.q.out
index 0d2fc21..f24b14d 100644
--- a/ql/src/test/results/clientpositive/perf/query64.q.out
+++ b/ql/src/test/results/clientpositive/perf/query64.q.out
@@ -1,6 +1,238 @@
-PREHOOK: query: explain select cs1.product_name ,cs1.store_name ,cs1.store_zip ,cs1.b_street_number ,cs1.b_streen_name ,cs1.b_city ,cs1.b_zip ,cs1.c_street_number ,cs1.c_street_name ,cs1.c_city ,cs1.c_zip ,cs1.syear ,cs1.cnt ,cs1.s1 ,cs1.s2 ,cs1.s3 ,cs2.s1 ,cs2.s2 ,cs2.s3 ,cs2.syear ,cs2.cnt from (select i_product_name as product_name ,i_item_sk as item_sk ,s_store_name as store_name ,s_zip as store_zip ,ad1.ca_street_number as b_street_number ,ad1.ca_street_name as b_streen_name ,ad1.ca_city as b_city ,ad1.ca_zip as b_zip ,ad2.ca_street_number as c_street_number ,ad2.ca_street_name as c_street_name ,ad2.ca_city as c_city ,ad2.ca_zip as c_zip ,d1.d_year as syear ,d2.d_year as fsyear ,d3.d_year as s2year ,count(*) as cnt ,sum(ss_wholesale_cost) as s1 ,sum(ss_list_price) as s2 ,sum(ss_coupon_amt) as s3 FROM store_sales JOIN store_returns ON store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number JOIN customer ON store_sales.s
 s_customer_sk = customer.c_customer_sk JOIN date_dim d1 ON store_sales.ss_sold_date_sk = d1.d_date_sk JOIN date_dim d2 ON customer.c_first_sales_date_sk = d2.d_date_sk JOIN date_dim d3 ON customer.c_first_shipto_date_sk = d3.d_date_sk JOIN store ON store_sales.ss_store_sk = store.s_store_sk JOIN customer_demographics cd1 ON store_sales.ss_cdemo_sk= cd1.cd_demo_sk JOIN customer_demographics cd2 ON customer.c_current_cdemo_sk = cd2.cd_demo_sk JOIN promotion ON store_sales.ss_promo_sk = promotion.p_promo_sk JOIN household_demographics hd1 ON store_sales.ss_hdemo_sk = hd1.hd_demo_sk JOIN household_demographics hd2 ON customer.c_current_hdemo_sk = hd2.hd_demo_sk JOIN customer_address ad1 ON store_sales.ss_addr_sk = ad1.ca_address_sk JOIN customer_address ad2 ON customer.c_current_addr_sk = ad2.ca_address_sk JOIN income_band ib1 ON hd1.hd_income_band_sk = ib1.ib_income_band_sk JOIN income_band ib2 ON hd2.hd_income_band_sk = ib2.ib_income_band_sk JOIN item ON store_sales.ss_item_sk = item.
 i_item_sk JOIN (select cs_item_sk ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund from catalog_sales JOIN catalog_returns ON catalog_sales.cs_item_sk = catalog_returns.cr_item_sk and catalog_sales.cs_order_number = catalog_returns.cr_order_number group by cs_item_sk having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)) cs_ui ON store_sales.ss_item_sk = cs_ui.cs_item_sk WHERE cd1.cd_marital_status <> cd2.cd_marital_status and i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and i_current_price between 35 and 35 + 10 and i_current_price between 35 + 1 and 35 + 15 group by i_product_name ,i_item_sk ,s_store_name ,s_zip ,ad1.ca_street_number ,ad1.ca_street_name ,ad1.ca_city ,ad1.ca_zip ,ad2.ca_street_number ,ad2.ca_street_name ,ad2.ca_city ,ad2.ca_zip ,d1.d_year ,d2.d_year ,d3.d_year ) cs1 JOIN (select i_product_name as product_name ,i_item_sk as item_sk ,s_store_name as store_name
  ,s_zip as store_zip ,ad1.ca_street_number as b_street_number ,ad1.ca_street_name as b_streen_name ,ad1.ca_city as b_city ,ad1.ca_zip as b_zip ,ad2.ca_street_number as c_street_number ,ad2.ca_street_name as c_street_name ,ad2.ca_city as c_city ,ad2.ca_zip as c_zip ,d1.d_year as syear ,d2.d_year as fsyear ,d3.d_year as s2year ,count(*) as cnt ,sum(ss_wholesale_cost) as s1 ,sum(ss_list_price) as s2 ,sum(ss_coupon_amt) as s3 FROM store_sales JOIN store_returns ON store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number JOIN customer ON store_sales.ss_customer_sk = customer.c_customer_sk JOIN date_dim d1 ON store_sales.ss_sold_date_sk = d1.d_date_sk JOIN date_dim d2 ON customer.c_first_sales_date_sk = d2.d_date_sk JOIN date_dim d3 ON customer.c_first_shipto_date_sk = d3.d_date_sk JOIN store ON store_sales.ss_store_sk = store.s_store_sk JOIN customer_demographics cd1 ON store_sales.ss_cdemo_sk= cd1.cd_demo_sk JOIN customer_demogr
 aphics cd2 ON customer.c_current_cdemo_sk = cd2.cd_demo_sk JOIN promotion ON store_sales.ss_promo_sk = promotion.p_promo_sk JOIN household_demographics hd1 ON store_sales.ss_hdemo_sk = hd1.hd_demo_sk JOIN household_demographics hd2 ON customer.c_current_hdemo_sk = hd2.hd_demo_sk JOIN customer_address ad1 ON store_sales.ss_addr_sk = ad1.ca_address_sk JOIN customer_address ad2 ON customer.c_current_addr_sk = ad2.ca_address_sk JOIN income_band ib1 ON hd1.hd_income_band_sk = ib1.ib_income_band_sk JOIN income_band ib2 ON hd2.hd_income_band_sk = ib2.ib_income_band_sk JOIN item ON store_sales.ss_item_sk = item.i_item_sk JOIN (select cs_item_sk ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund from catalog_sales JOIN catalog_returns ON catalog_sales.cs_item_sk = catalog_returns.cr_item_sk and catalog_sales.cs_order_number = catalog_returns.cr_order_number group by cs_item_sk having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_cha
 rge+cr_store_credit)) cs_ui ON store_sales.ss_item_sk = cs_ui.cs_item_sk WHERE cd1.cd_marital_status <> cd2.cd_marital_status and i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and i_current_price between 35 and 35 + 10 and i_current_price between 35 + 1 and 35 + 15 group by i_product_name ,i_item_sk ,s_store_name ,s_zip ,ad1.ca_street_number ,ad1.ca_street_name ,ad1.ca_city ,ad1.ca_zip ,ad2.ca_street_number ,ad2.ca_street_name ,ad2.ca_city ,ad2.ca_zip ,d1.d_year ,d2.d_year ,d3.d_year ) cs2 ON cs1.item_sk=cs2.item_sk where cs1.syear = 2000 and cs2.syear = 2000 + 1 and cs2.cnt <= cs1.cnt and cs1.store_name = cs2.store_name and cs1.store_zip = cs2.store_zip order by cs1.product_name ,cs1.store_name ,cs2.cnt
+PREHOOK: query: explain
+with cs_ui as
+ (select cs_item_sk
+        ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund
+  from catalog_sales
+      ,catalog_returns
+  where cs_item_sk = cr_item_sk
+    and cs_order_number = cr_order_number
+  group by cs_item_sk
+  having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)),
+cross_sales as
+ (select i_product_name product_name
+     ,i_item_sk item_sk
+     ,s_store_name store_name
+     ,s_zip store_zip
+     ,ad1.ca_street_number b_street_number
+     ,ad1.ca_street_name b_streen_name
+     ,ad1.ca_city b_city
+     ,ad1.ca_zip b_zip
+     ,ad2.ca_street_number c_street_number
+     ,ad2.ca_street_name c_street_name
+     ,ad2.ca_city c_city
+     ,ad2.ca_zip c_zip
+     ,d1.d_year as syear
+     ,d2.d_year as fsyear
+     ,d3.d_year s2year
+     ,count(*) cnt
+     ,sum(ss_wholesale_cost) s1
+     ,sum(ss_list_price) s2
+     ,sum(ss_coupon_amt) s3
+  FROM   store_sales
+        ,store_returns
+        ,cs_ui
+        ,date_dim d1
+        ,date_dim d2
+        ,date_dim d3
+        ,store
+        ,customer
+        ,customer_demographics cd1
+        ,customer_demographics cd2
+        ,promotion
+        ,household_demographics hd1
+        ,household_demographics hd2
+        ,customer_address ad1
+        ,customer_address ad2
+        ,income_band ib1
+        ,income_band ib2
+        ,item
+  WHERE  ss_store_sk = s_store_sk AND
+         ss_sold_date_sk = d1.d_date_sk AND
+         ss_customer_sk = c_customer_sk AND
+         ss_cdemo_sk= cd1.cd_demo_sk AND
+         ss_hdemo_sk = hd1.hd_demo_sk AND
+         ss_addr_sk = ad1.ca_address_sk and
+         ss_item_sk = i_item_sk and
+         ss_item_sk = sr_item_sk and
+         ss_ticket_number = sr_ticket_number and
+         ss_item_sk = cs_ui.cs_item_sk and
+         c_current_cdemo_sk = cd2.cd_demo_sk AND
+         c_current_hdemo_sk = hd2.hd_demo_sk AND
+         c_current_addr_sk = ad2.ca_address_sk and
+         c_first_sales_date_sk = d2.d_date_sk and
+         c_first_shipto_date_sk = d3.d_date_sk and
+         ss_promo_sk = p_promo_sk and
+         hd1.hd_income_band_sk = ib1.ib_income_band_sk and
+         hd2.hd_income_band_sk = ib2.ib_income_band_sk and
+         cd1.cd_marital_status <> cd2.cd_marital_status and
+         i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and
+         i_current_price between 35 and 35 + 10 and
+         i_current_price between 35 + 1 and 35 + 15
+group by i_product_name
+       ,i_item_sk
+       ,s_store_name
+       ,s_zip
+       ,ad1.ca_street_number
+       ,ad1.ca_street_name
+       ,ad1.ca_city
+       ,ad1.ca_zip
+       ,ad2.ca_street_number
+       ,ad2.ca_street_name
+       ,ad2.ca_city
+       ,ad2.ca_zip
+       ,d1.d_year
+       ,d2.d_year
+       ,d3.d_year
+)
+select cs1.product_name
+     ,cs1.store_name
+     ,cs1.store_zip
+     ,cs1.b_street_number
+     ,cs1.b_streen_name
+     ,cs1.b_city
+     ,cs1.b_zip
+     ,cs1.c_street_number
+     ,cs1.c_street_name
+     ,cs1.c_city
+     ,cs1.c_zip
+     ,cs1.syear
+     ,cs1.cnt
+     ,cs1.s1
+     ,cs1.s2
+     ,cs1.s3
+     ,cs2.s1
+     ,cs2.s2
+     ,cs2.s3
+     ,cs2.syear
+     ,cs2.cnt
+from cross_sales cs1,cross_sales cs2
+where cs1.item_sk=cs2.item_sk and
+     cs1.syear = 2000 and
+     cs2.syear = 2000 + 1 and
+     cs2.cnt <= cs1.cnt and
+     cs1.store_name = cs2.store_name and
+     cs1.store_zip = cs2.store_zip
+order by cs1.product_name
+       ,cs1.store_name
+       ,cs2.cnt
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select cs1.product_name ,cs1.store_name ,cs1.store_zip ,cs1.b_street_number ,cs1.b_streen_name ,cs1.b_city ,cs1.b_zip ,cs1.c_street_number ,cs1.c_street_name ,cs1.c_city ,cs1.c_zip ,cs1.syear ,cs1.cnt ,cs1.s1 ,cs1.s2 ,cs1.s3 ,cs2.s1 ,cs2.s2 ,cs2.s3 ,cs2.syear ,cs2.cnt from (select i_product_name as product_name ,i_item_sk as item_sk ,s_store_name as store_name ,s_zip as store_zip ,ad1.ca_street_number as b_street_number ,ad1.ca_street_name as b_streen_name ,ad1.ca_city as b_city ,ad1.ca_zip as b_zip ,ad2.ca_street_number as c_street_number ,ad2.ca_street_name as c_street_name ,ad2.ca_city as c_city ,ad2.ca_zip as c_zip ,d1.d_year as syear ,d2.d_year as fsyear ,d3.d_year as s2year ,count(*) as cnt ,sum(ss_wholesale_cost) as s1 ,sum(ss_list_price) as s2 ,sum(ss_coupon_amt) as s3 FROM store_sales JOIN store_returns ON store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number JOIN customer ON store_sales.
 ss_customer_sk = customer.c_customer_sk JOIN date_dim d1 ON store_sales.ss_sold_date_sk = d1.d_date_sk JOIN date_dim d2 ON customer.c_first_sales_date_sk = d2.d_date_sk JOIN date_dim d3 ON customer.c_first_shipto_date_sk = d3.d_date_sk JOIN store ON store_sales.ss_store_sk = store.s_store_sk JOIN customer_demographics cd1 ON store_sales.ss_cdemo_sk= cd1.cd_demo_sk JOIN customer_demographics cd2 ON customer.c_current_cdemo_sk = cd2.cd_demo_sk JOIN promotion ON store_sales.ss_promo_sk = promotion.p_promo_sk JOIN household_demographics hd1 ON store_sales.ss_hdemo_sk = hd1.hd_demo_sk JOIN household_demographics hd2 ON customer.c_current_hdemo_sk = hd2.hd_demo_sk JOIN customer_address ad1 ON store_sales.ss_addr_sk = ad1.ca_address_sk JOIN customer_address ad2 ON customer.c_current_addr_sk = ad2.ca_address_sk JOIN income_band ib1 ON hd1.hd_income_band_sk = ib1.ib_income_band_sk JOIN income_band ib2 ON hd2.hd_income_band_sk = ib2.ib_income_band_sk JOIN item ON store_sales.ss_item_sk = item
 .i_item_sk JOIN (select cs_item_sk ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund from catalog_sales JOIN catalog_returns ON catalog_sales.cs_item_sk = catalog_returns.cr_item_sk and catalog_sales.cs_order_number = catalog_returns.cr_order_number group by cs_item_sk having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)) cs_ui ON store_sales.ss_item_sk = cs_ui.cs_item_sk WHERE cd1.cd_marital_status <> cd2.cd_marital_status and i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and i_current_price between 35 and 35 + 10 and i_current_price between 35 + 1 and 35 + 15 group by i_product_name ,i_item_sk ,s_store_name ,s_zip ,ad1.ca_street_number ,ad1.ca_street_name ,ad1.ca_city ,ad1.ca_zip ,ad2.ca_street_number ,ad2.ca_street_name ,ad2.ca_city ,ad2.ca_zip ,d1.d_year ,d2.d_year ,d3.d_year ) cs1 JOIN (select i_product_name as product_name ,i_item_sk as item_sk ,s_store_name as store_nam
 e ,s_zip as store_zip ,ad1.ca_street_number as b_street_number ,ad1.ca_street_name as b_streen_name ,ad1.ca_city as b_city ,ad1.ca_zip as b_zip ,ad2.ca_street_number as c_street_number ,ad2.ca_street_name as c_street_name ,ad2.ca_city as c_city ,ad2.ca_zip as c_zip ,d1.d_year as syear ,d2.d_year as fsyear ,d3.d_year as s2year ,count(*) as cnt ,sum(ss_wholesale_cost) as s1 ,sum(ss_list_price) as s2 ,sum(ss_coupon_amt) as s3 FROM store_sales JOIN store_returns ON store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number JOIN customer ON store_sales.ss_customer_sk = customer.c_customer_sk JOIN date_dim d1 ON store_sales.ss_sold_date_sk = d1.d_date_sk JOIN date_dim d2 ON customer.c_first_sales_date_sk = d2.d_date_sk JOIN date_dim d3 ON customer.c_first_shipto_date_sk = d3.d_date_sk JOIN store ON store_sales.ss_store_sk = store.s_store_sk JOIN customer_demographics cd1 ON store_sales.ss_cdemo_sk= cd1.cd_demo_sk JOIN customer_demog
 raphics cd2 ON customer.c_current_cdemo_sk = cd2.cd_demo_sk JOIN promotion ON store_sales.ss_promo_sk = promotion.p_promo_sk JOIN household_demographics hd1 ON store_sales.ss_hdemo_sk = hd1.hd_demo_sk JOIN household_demographics hd2 ON customer.c_current_hdemo_sk = hd2.hd_demo_sk JOIN customer_address ad1 ON store_sales.ss_addr_sk = ad1.ca_address_sk JOIN customer_address ad2 ON customer.c_current_addr_sk = ad2.ca_address_sk JOIN income_band ib1 ON hd1.hd_income_band_sk = ib1.ib_income_band_sk JOIN income_band ib2 ON hd2.hd_income_band_sk = ib2.ib_income_band_sk JOIN item ON store_sales.ss_item_sk = item.i_item_sk JOIN (select cs_item_sk ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund from catalog_sales JOIN catalog_returns ON catalog_sales.cs_item_sk = catalog_returns.cr_item_sk and catalog_sales.cs_order_number = catalog_returns.cr_order_number group by cs_item_sk having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_ch
 arge+cr_store_credit)) cs_ui ON store_sales.ss_item_sk = cs_ui.cs_item_sk WHERE cd1.cd_marital_status <> cd2.cd_marital_status and i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and i_current_price between 35 and 35 + 10 and i_current_price between 35 + 1 and 35 + 15 group by i_product_name ,i_item_sk ,s_store_name ,s_zip ,ad1.ca_street_number ,ad1.ca_street_name ,ad1.ca_city ,ad1.ca_zip ,ad2.ca_street_number ,ad2.ca_street_name ,ad2.ca_city ,ad2.ca_zip ,d1.d_year ,d2.d_year ,d3.d_year ) cs2 ON cs1.item_sk=cs2.item_sk where cs1.syear = 2000 and cs2.syear = 2000 + 1 and cs2.cnt <= cs1.cnt and cs1.store_name = cs2.store_name and cs1.store_zip = cs2.store_zip order by cs1.product_name ,cs1.store_name ,cs2.cnt
+POSTHOOK: query: explain
+with cs_ui as
+ (select cs_item_sk
+        ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund
+  from catalog_sales
+      ,catalog_returns
+  where cs_item_sk = cr_item_sk
+    and cs_order_number = cr_order_number
+  group by cs_item_sk
+  having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)),
+cross_sales as
+ (select i_product_name product_name
+     ,i_item_sk item_sk
+     ,s_store_name store_name
+     ,s_zip store_zip
+     ,ad1.ca_street_number b_street_number
+     ,ad1.ca_street_name b_streen_name
+     ,ad1.ca_city b_city
+     ,ad1.ca_zip b_zip
+     ,ad2.ca_street_number c_street_number
+     ,ad2.ca_street_name c_street_name
+     ,ad2.ca_city c_city
+     ,ad2.ca_zip c_zip
+     ,d1.d_year as syear
+     ,d2.d_year as fsyear
+     ,d3.d_year s2year
+     ,count(*) cnt
+     ,sum(ss_wholesale_cost) s1
+     ,sum(ss_list_price) s2
+     ,sum(ss_coupon_amt) s3
+  FROM   store_sales
+        ,store_returns
+        ,cs_ui
+        ,date_dim d1
+        ,date_dim d2
+        ,date_dim d3
+        ,store
+        ,customer
+        ,customer_demographics cd1
+        ,customer_demographics cd2
+        ,promotion
+        ,household_demographics hd1
+        ,household_demographics hd2
+        ,customer_address ad1
+        ,customer_address ad2
+        ,income_band ib1
+        ,income_band ib2
+        ,item
+  WHERE  ss_store_sk = s_store_sk AND
+         ss_sold_date_sk = d1.d_date_sk AND
+         ss_customer_sk = c_customer_sk AND
+         ss_cdemo_sk= cd1.cd_demo_sk AND
+         ss_hdemo_sk = hd1.hd_demo_sk AND
+         ss_addr_sk = ad1.ca_address_sk and
+         ss_item_sk = i_item_sk and
+         ss_item_sk = sr_item_sk and
+         ss_ticket_number = sr_ticket_number and
+         ss_item_sk = cs_ui.cs_item_sk and
+         c_current_cdemo_sk = cd2.cd_demo_sk AND
+         c_current_hdemo_sk = hd2.hd_demo_sk AND
+         c_current_addr_sk = ad2.ca_address_sk and
+         c_first_sales_date_sk = d2.d_date_sk and
+         c_first_shipto_date_sk = d3.d_date_sk and
+         ss_promo_sk = p_promo_sk and
+         hd1.hd_income_band_sk = ib1.ib_income_band_sk and
+         hd2.hd_income_band_sk = ib2.ib_income_band_sk and
+         cd1.cd_marital_status <> cd2.cd_marital_status and
+         i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and
+         i_current_price between 35 and 35 + 10 and
+         i_current_price between 35 + 1 and 35 + 15
+group by i_product_name
+       ,i_item_sk
+       ,s_store_name
+       ,s_zip
+       ,ad1.ca_street_number
+       ,ad1.ca_street_name
+       ,ad1.ca_city
+       ,ad1.ca_zip
+       ,ad2.ca_street_number
+       ,ad2.ca_street_name
+       ,ad2.ca_city
+       ,ad2.ca_zip
+       ,d1.d_year
+       ,d2.d_year
+       ,d3.d_year
+)
+select cs1.product_name
+     ,cs1.store_name
+     ,cs1.store_zip
+     ,cs1.b_street_number
+     ,cs1.b_streen_name
+     ,cs1.b_city
+     ,cs1.b_zip
+     ,cs1.c_street_number
+     ,cs1.c_street_name
+     ,cs1.c_city
+     ,cs1.c_zip
+     ,cs1.syear
+     ,cs1.cnt
+     ,cs1.s1
+     ,cs1.s2
+     ,cs1.s3
+     ,cs2.s1
+     ,cs2.s2
+     ,cs2.s3
+     ,cs2.syear
+     ,cs2.cnt
+from cross_sales cs1,cross_sales cs2
+where cs1.item_sk=cs2.item_sk and
+     cs1.syear = 2000 and
+     cs2.syear = 2000 + 1 and
+     cs2.cnt <= cs1.cnt and
+     cs1.store_name = cs2.store_name and
+     cs1.store_zip = cs2.store_zip
+order by cs1.product_name
+       ,cs1.store_name
+       ,cs2.cnt
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -161,7 +393,7 @@ Stage-0
                                                             Select Operator [SEL_130] (rows=80000000 width=860)
                                                               Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                                                               Filter Operator [FIL_556] (rows=80000000 width=860)
-                                                                predicate:(c_customer_sk is not null and c_first_sales_date_sk is not null and c_first_shipto_date_sk is not null and c_current_cdemo_sk is not null and c_current_hdemo_sk is not null and c_current_addr_sk is not null)
+                                                                predicate:(c_customer_sk is not null and c_first_shipto_date_sk is not null and c_first_sales_date_sk is not null and c_current_cdemo_sk is not null and c_current_hdemo_sk is not null and c_current_addr_sk is not null)
                                                                 TableScan [TS_0] (rows=80000000 width=860)
                                                                   default@customer,customer,Tbl:COMPLETE,Col:NONE,Output:["c_customer_sk","c_current_cdemo_sk","c_current_hdemo_sk","c_current_addr_sk","c_first_shipto_date_sk","c_first_sales_date_sk"]
                                                 <-Reducer 36 [SIMPLE_EDGE]
@@ -280,7 +512,7 @@ Stage-0
                                                                             Select Operator [SEL_161] (rows=575995635 width=88)
                                                                               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                                                               Filter Operator [FIL_565] (rows=575995635 width=88)
-                                                                                predicate:(ss_item_sk is not null and ss_ticket_number is not null and ss_customer_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null and ss_cdemo_sk is not null and ss_promo_sk is not null and ss_hdemo_sk is not null and ss_addr_sk is not null)
+                                                                                predicate:(ss_item_sk is not null and ss_ticket_number is not null and ss_sold_date_sk is not null and ss_store_sk is not null and ss_customer_sk is not null and ss_cdemo_sk is not null and ss_promo_sk is not null and ss_hdemo_sk is not null and ss_addr_sk is not null)
                                                                                 TableScan [TS_31] (rows=575995635 width=88)
                                                                                   default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_cdemo_sk","ss_hdemo_sk","ss_addr_sk","ss_store_sk","ss_promo_sk","ss_ticket_number","ss_wholesale_cost","ss_list_price","ss_coupon_amt"]
                                                                         <-Map 46 [SIMPLE_EDGE]
@@ -470,7 +702,7 @@ Stage-0
                                                                             Select Operator [SEL_33] (rows=575995635 width=88)
                                                                               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                                                               Filter Operator [FIL_546] (rows=575995635 width=88)
-                                                                                predicate:(ss_item_sk is not null and ss_ticket_number is not null and ss_customer_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null and ss_cdemo_sk is not null and ss_promo_sk is not null and ss_hdemo_sk is not null and ss_addr_sk is not null)
+                                                                                predicate:(ss_item_sk is not null and ss_ticket_number is not null and ss_sold_date_sk is not null and ss_store_sk is not null and ss_customer_sk is not null and ss_cdemo_sk is not null and ss_promo_sk is not null and ss_hdemo_sk is not null and ss_addr_sk is not null)
                                                                                  Please refer to the previous TableScan [TS_31]
                                                                         <-Map 46 [SIMPLE_EDGE]
                                                                           SHUFFLE [RS_47]
@@ -598,7 +830,7 @@ Stage-0
                                                             Select Operator [SEL_2] (rows=80000000 width=860)
                                                               Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                                                               Filter Operator [FIL_537] (rows=80000000 width=860)
-                                                                predicate:(c_customer_sk is not null and c_first_sales_date_sk is not null and c_first_shipto_date_sk is not null and c_current_cdemo_sk is not null and c_current_hdemo_sk is not null and c_current_addr_sk is not null)
+                                                                predicate:(c_customer_sk is not null and c_first_shipto_date_sk is not null and c_first_sales_date_sk is not null and c_current_cdemo_sk is not null and c_current_hdemo_sk is not null and c_current_addr_sk is not null)
                                                                  Please refer to the previous TableScan [TS_0]
                                                 <-Reducer 34 [SIMPLE_EDGE]
                                                   SHUFFLE [RS_107]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query65.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query65.q.out b/ql/src/test/results/clientpositive/perf/query65.q.out
index 17d80d0..b2035c2 100644
--- a/ql/src/test/results/clientpositive/perf/query65.q.out
+++ b/ql/src/test/results/clientpositive/perf/query65.q.out
@@ -1,77 +1,57 @@
-PREHOOK: query: explain select 
-    s_store_name,
-    i_item_desc,
-    sc.revenue,
-    i_current_price,
-    i_wholesale_cost,
-    i_brand
-from
-    store,
-    item,
-    (select 
-        ss_store_sk, avg(revenue) as ave
-    from
-        (select 
-        ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
-    from
-        store_sales, date_dim
-    where
-        ss_sold_date_sk = d_date_sk
-            and d_month_seq between 1212 and 1212 + 11
-    group by ss_store_sk , ss_item_sk) sa
-    group by ss_store_sk) sb,
-    (select 
-        ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
-    from
-        store_sales, date_dim
-    where
-        ss_sold_date_sk = d_date_sk
-            and d_month_seq between 1212 and 1212 + 11
-    group by ss_store_sk , ss_item_sk) sc
-where
-    sb.ss_store_sk = sc.ss_store_sk
-        and sc.revenue <= 0.1 * sb.ave
-        and s_store_sk = sc.ss_store_sk
-        and i_item_sk = sc.ss_item_sk
-order by s_store_name , i_item_desc
+PREHOOK: query: explain
+select 
+	s_store_name,
+	i_item_desc,
+	sc.revenue,
+	i_current_price,
+	i_wholesale_cost,
+	i_brand
+ from store, item,
+     (select ss_store_sk, avg(revenue) as ave
+ 	from
+ 	    (select  ss_store_sk, ss_item_sk, 
+ 		     sum(ss_sales_price) as revenue
+ 		from store_sales, date_dim
+ 		where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11
+ 		group by ss_store_sk, ss_item_sk) sa
+ 	group by ss_store_sk) sb,
+     (select  ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
+ 	from store_sales, date_dim
+ 	where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11
+ 	group by ss_store_sk, ss_item_sk) sc
+ where sb.ss_store_sk = sc.ss_store_sk and 
+       sc.revenue <= 0.1 * sb.ave and
+       s_store_sk = sc.ss_store_sk and
+       i_item_sk = sc.ss_item_sk
+ order by s_store_name, i_item_desc
 limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select 
-    s_store_name,
-    i_item_desc,
-    sc.revenue,
-    i_current_price,
-    i_wholesale_cost,
-    i_brand
-from
-    store,
-    item,
-    (select 
-        ss_store_sk, avg(revenue) as ave
-    from
-        (select 
-        ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
-    from
-        store_sales, date_dim
-    where
-        ss_sold_date_sk = d_date_sk
-            and d_month_seq between 1212 and 1212 + 11
-    group by ss_store_sk , ss_item_sk) sa
-    group by ss_store_sk) sb,
-    (select 
-        ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
-    from
-        store_sales, date_dim
-    where
-        ss_sold_date_sk = d_date_sk
-            and d_month_seq between 1212 and 1212 + 11
-    group by ss_store_sk , ss_item_sk) sc
-where
-    sb.ss_store_sk = sc.ss_store_sk
-        and sc.revenue <= 0.1 * sb.ave
-        and s_store_sk = sc.ss_store_sk
-        and i_item_sk = sc.ss_item_sk
-order by s_store_name , i_item_desc
+POSTHOOK: query: explain
+select 
+	s_store_name,
+	i_item_desc,
+	sc.revenue,
+	i_current_price,
+	i_wholesale_cost,
+	i_brand
+ from store, item,
+     (select ss_store_sk, avg(revenue) as ave
+ 	from
+ 	    (select  ss_store_sk, ss_item_sk, 
+ 		     sum(ss_sales_price) as revenue
+ 		from store_sales, date_dim
+ 		where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11
+ 		group by ss_store_sk, ss_item_sk) sa
+ 	group by ss_store_sk) sb,
+     (select  ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
+ 	from store_sales, date_dim
+ 	where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11
+ 	group by ss_store_sk, ss_item_sk) sc
+ where sb.ss_store_sk = sc.ss_store_sk and 
+       sc.revenue <= 0.1 * sb.ave and
+       s_store_sk = sc.ss_store_sk and
+       i_item_sk = sc.ss_item_sk
+ order by s_store_name, i_item_desc
 limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/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 19cd0fb..2c74815 100644
--- a/ql/src/test/results/clientpositive/perf/query66.q.out
+++ b/ql/src/test/results/clientpositive/perf/query66.q.out
@@ -1,11 +1,11 @@
 PREHOOK: query: explain
 select   
          w_warehouse_name
- 	 ,w_warehouse_sq_ft
- 	 ,w_city
- 	 ,w_county
- 	 ,w_state
- 	 ,w_country
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
         ,ship_carriers
         ,year
  	,sum(jan_sales) as jan_sales
@@ -45,74 +45,74 @@ select
  	,sum(nov_net) as nov_net
  	,sum(dec_net) as dec_net
  from (
-    select 
-    w_warehouse_name
-    ,w_warehouse_sq_ft
-    ,w_city
-    ,w_county
-    ,w_state
-    ,w_country
-    ,concat('DIAMOND', ',', 'AIRBORNE') as ship_carriers
-        ,d_year as year
+    (select 
+ 	w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+ 	,'DIAMOND' || ',' || 'AIRBORNE' as ship_carriers
+       ,d_year as year
  	,sum(case when d_moy = 1 
- 		  then ws_sales_price* ws_quantity else 0 end) as jan_sales
- 		  ,sum(case when d_moy = 2 
- 		  	    then ws_sales_price* ws_quantity else 0 end) as feb_sales
- 			    ,sum(case when d_moy = 3 
- 			    	      then ws_sales_price* ws_quantity else 0 end) as mar_sales
- 				      ,sum(case when d_moy = 4 
- 				      		then ws_sales_price* ws_quantity else 0 end) as apr_sales
- 						,sum(case when d_moy = 5 
- 							  then ws_sales_price* ws_quantity else 0 end) as may_sales
- 							  ,sum(case when d_moy = 6 
- 							  	    then ws_sales_price* ws_quantity else 0 end) as jun_sales
- 								    ,sum(case when d_moy = 7 
- 								    	      then ws_sales_price* ws_quantity else 0 end) as jul_sales
- 									      ,sum(case when d_moy = 8 
- 									      		then ws_sales_price* ws_quantity else 0 end) as aug_sales
- 											,sum(case when d_moy = 9 
- 												  then ws_sales_price* ws_quantity else 0 end) as sep_sales
- 												  ,sum(case when d_moy = 10 
- 												  	    then ws_sales_price* ws_quantity else 0 end) as oct_sales
- 													    ,sum(case when d_moy = 11
- 													    	      then ws_sales_price* ws_quantity else 0 end) as nov_sales
- 														      ,sum(case when d_moy = 12
- 														      		then ws_sales_price* ws_quantity else 0 end) as dec_sales
- 																,sum(case when d_moy = 1 
- 																	  then ws_net_paid_inc_tax * ws_quantity else 0 end) as jan_net
- 																	  ,sum(case when d_moy = 2
- 																	  	    then ws_net_paid_inc_tax * ws_quantity else 0 end) as feb_net
- 																		    ,sum(case when d_moy = 3 
- 																		    	      then ws_net_paid_inc_tax * ws_quantity else 0 end) as mar_net
- 																			      ,sum(case when d_moy = 4 
- 																			      		then ws_net_paid_inc_tax * ws_quantity else 0 end) as apr_net
- 																					,sum(case when d_moy = 5 
- 																						  then ws_net_paid_inc_tax * ws_quantity else 0 end) as may_net
- 																						  ,sum(case when d_moy = 6 
- 																						  	    then ws_net_paid_inc_tax * ws_quantity else 0 end) as jun_net
- 																							    ,sum(case when d_moy = 7 
- 																							    	      then ws_net_paid_inc_tax * ws_quantity else 0 end) as jul_net
- 																								      ,sum(case when d_moy = 8 
- 																								      		then ws_net_paid_inc_tax * ws_quantity else 0 end) as aug_net
- 																										,sum(case when d_moy = 9 
- 																											  then ws_net_paid_inc_tax * ws_quantity else 0 end) as sep_net
- 																											  ,sum(case when d_moy = 10 
- 																											  	    then ws_net_paid_inc_tax * ws_quantity else 0 end) as oct_net
- 																												    ,sum(case when d_moy = 11
- 																												    	      then ws_net_paid_inc_tax * ws_quantity else 0 end) as nov_net
- 																													      ,sum(case when d_moy = 12
- 																													      		then ws_net_paid_inc_tax * ws_quantity else 0 end) as dec_net
+ 		then ws_sales_price* ws_quantity else 0 end) as jan_sales
+ 	,sum(case when d_moy = 2 
+ 		then ws_sales_price* ws_quantity else 0 end) as feb_sales
+ 	,sum(case when d_moy = 3 
+ 		then ws_sales_price* ws_quantity else 0 end) as mar_sales
+ 	,sum(case when d_moy = 4 
+ 		then ws_sales_price* ws_quantity else 0 end) as apr_sales
+ 	,sum(case when d_moy = 5 
+ 		then ws_sales_price* ws_quantity else 0 end) as may_sales
+ 	,sum(case when d_moy = 6 
+ 		then ws_sales_price* ws_quantity else 0 end) as jun_sales
+ 	,sum(case when d_moy = 7 
+ 		then ws_sales_price* ws_quantity else 0 end) as jul_sales
+ 	,sum(case when d_moy = 8 
+ 		then ws_sales_price* ws_quantity else 0 end) as aug_sales
+ 	,sum(case when d_moy = 9 
+ 		then ws_sales_price* ws_quantity else 0 end) as sep_sales
+ 	,sum(case when d_moy = 10 
+ 		then ws_sales_price* ws_quantity else 0 end) as oct_sales
+ 	,sum(case when d_moy = 11
+ 		then ws_sales_price* ws_quantity else 0 end) as nov_sales
+ 	,sum(case when d_moy = 12
+ 		then ws_sales_price* ws_quantity else 0 end) as dec_sales
+ 	,sum(case when d_moy = 1 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jan_net
+ 	,sum(case when d_moy = 2
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as feb_net
+ 	,sum(case when d_moy = 3 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as mar_net
+ 	,sum(case when d_moy = 4 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as apr_net
+ 	,sum(case when d_moy = 5 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as may_net
+ 	,sum(case when d_moy = 6 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jun_net
+ 	,sum(case when d_moy = 7 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jul_net
+ 	,sum(case when d_moy = 8 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as aug_net
+ 	,sum(case when d_moy = 9 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as sep_net
+ 	,sum(case when d_moy = 10 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as oct_net
+ 	,sum(case when d_moy = 11
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as nov_net
+ 	,sum(case when d_moy = 12
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as dec_net
      from
           web_sales
          ,warehouse
          ,date_dim
          ,time_dim
- 	   ,ship_mode
+ 	  ,ship_mode
      where
-            web_sales.ws_warehouse_sk =  warehouse.w_warehouse_sk
-        and web_sales.ws_sold_date_sk = date_dim.d_date_sk
-        and web_sales.ws_sold_time_sk = time_dim.t_time_sk
- 	and web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk
+            ws_warehouse_sk =  w_warehouse_sk
+        and ws_sold_date_sk = d_date_sk
+        and ws_sold_time_sk = t_time_sk
+ 	and ws_ship_mode_sk = sm_ship_mode_sk
         and d_year = 2002
  	and t_time between 49530 and 49530+28800 
  	and sm_carrier in ('DIAMOND','AIRBORNE')
@@ -124,75 +124,76 @@ select
  	,w_state
  	,w_country
        ,d_year
+ 	)
  union all
-    select 
-    w_warehouse_name
-    ,w_warehouse_sq_ft
-    ,w_city
-    ,w_county
-    ,w_state
-    ,w_country
-        ,concat('DIAMOND', ',', 'AIRBORNE') as ship_carriers
+    (select 
+ 	w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+ 	,'DIAMOND' || ',' || 'AIRBORNE' as ship_carriers
        ,d_year as year
-       ,sum(case when d_moy = 1 
-       		 then cs_ext_sales_price* cs_quantity else 0 end) as jan_sales
- 		 ,sum(case when d_moy = 2 
- 		 	   then cs_ext_sales_price* cs_quantity else 0 end) as feb_sales
- 			   ,sum(case when d_moy = 3 
- 			   	     then cs_ext_sales_price* cs_quantity else 0 end) as mar_sales
- 				     ,sum(case when d_moy = 4 
- 				     	       then cs_ext_sales_price* cs_quantity else 0 end) as apr_sales
- 					       ,sum(case when d_moy = 5 
- 					       		 then cs_ext_sales_price* cs_quantity else 0 end) as may_sales
- 							 ,sum(case when d_moy = 6 
- 							 	   then cs_ext_sales_price* cs_quantity else 0 end) as jun_sales
- 								   ,sum(case when d_moy = 7 
- 								   	     then cs_ext_sales_price* cs_quantity else 0 end) as jul_sales
- 									     ,sum(case when d_moy = 8 
- 									     	       then cs_ext_sales_price* cs_quantity else 0 end) as aug_sales
- 										       ,sum(case when d_moy = 9 
- 										       		 then cs_ext_sales_price* cs_quantity else 0 end) as sep_sales
- 												 ,sum(case when d_moy = 10 
- 												 	   then cs_ext_sales_price* cs_quantity else 0 end) as oct_sales
- 													   ,sum(case when d_moy = 11
- 													   	     then cs_ext_sales_price* cs_quantity else 0 end) as nov_sales
- 														     ,sum(case when d_moy = 12
- 														     	       then cs_ext_sales_price* cs_quantity else 0 end) as dec_sales
- 															       ,sum(case when d_moy = 1 
- 															       		 then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jan_net
- 																	 ,sum(case when d_moy = 2 
- 																	 	   then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as feb_net
- 																		   ,sum(case when d_moy = 3 
- 																		   	     then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as mar_net
- 																			     ,sum(case when d_moy = 4 
- 																			     	       then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as apr_net
- 																				       ,sum(case when d_moy = 5 
- 																				       		 then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as may_net
- 																						 ,sum(case when d_moy = 6 
- 																						 	   then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jun_net
- 																							   ,sum(case when d_moy = 7 
- 																							   	     then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jul_net
- 																								     ,sum(case when d_moy = 8 
- 																								     	       then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as aug_net
- 																									       ,sum(case when d_moy = 9 
- 																									       		 then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as sep_net
- 																											 ,sum(case when d_moy = 10 
- 																											 	   then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as oct_net
- 																												   ,sum(case when d_moy = 11
- 																												   	     then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as nov_net
- 																													     ,sum(case when d_moy = 12
- 																													     	       then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as dec_net
+ 	,sum(case when d_moy = 1 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jan_sales
+ 	,sum(case when d_moy = 2 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as feb_sales
+ 	,sum(case when d_moy = 3 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as mar_sales
+ 	,sum(case when d_moy = 4 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as apr_sales
+ 	,sum(case when d_moy = 5 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as may_sales
+ 	,sum(case when d_moy = 6 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jun_sales
+ 	,sum(case when d_moy = 7 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jul_sales
+ 	,sum(case when d_moy = 8 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as aug_sales
+ 	,sum(case when d_moy = 9 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as sep_sales
+ 	,sum(case when d_moy = 10 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as oct_sales
+ 	,sum(case when d_moy = 11
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as nov_sales
+ 	,sum(case when d_moy = 12
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as dec_sales
+ 	,sum(case when d_moy = 1 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jan_net
+ 	,sum(case when d_moy = 2 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as feb_net
+ 	,sum(case when d_moy = 3 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as mar_net
+ 	,sum(case when d_moy = 4 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as apr_net
+ 	,sum(case when d_moy = 5 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as may_net
+ 	,sum(case when d_moy = 6 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jun_net
+ 	,sum(case when d_moy = 7 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jul_net
+ 	,sum(case when d_moy = 8 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as aug_net
+ 	,sum(case when d_moy = 9 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as sep_net
+ 	,sum(case when d_moy = 10 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as oct_net
+ 	,sum(case when d_moy = 11
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as nov_net
+ 	,sum(case when d_moy = 12
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as dec_net
      from
           catalog_sales
          ,warehouse
          ,date_dim
          ,time_dim
- 	  ,ship_mode
+ 	 ,ship_mode
      where
-            catalog_sales.cs_warehouse_sk =  warehouse.w_warehouse_sk
-        and catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
-        and catalog_sales.cs_sold_time_sk = time_dim.t_time_sk
- 	and catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk
+            cs_warehouse_sk =  w_warehouse_sk
+        and cs_sold_date_sk = d_date_sk
+        and cs_sold_time_sk = t_time_sk
+ 	and cs_ship_mode_sk = sm_ship_mode_sk
         and d_year = 2002
  	and t_time between 49530 AND 49530+28800 
  	and sm_carrier in ('DIAMOND','AIRBORNE')
@@ -204,6 +205,7 @@ select
  	,w_state
  	,w_country
        ,d_year
+     ) 
  ) x
  group by 
         w_warehouse_name
@@ -220,11 +222,11 @@ PREHOOK: type: QUERY
 POSTHOOK: query: explain
 select   
          w_warehouse_name
- 	 ,w_warehouse_sq_ft
- 	 ,w_city
- 	 ,w_county
- 	 ,w_state
- 	 ,w_country
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
         ,ship_carriers
         ,year
  	,sum(jan_sales) as jan_sales
@@ -264,74 +266,74 @@ select
  	,sum(nov_net) as nov_net
  	,sum(dec_net) as dec_net
  from (
-    select 
-    w_warehouse_name
-    ,w_warehouse_sq_ft
-    ,w_city
-    ,w_county
-    ,w_state
-    ,w_country
-    ,concat('DIAMOND', ',', 'AIRBORNE') as ship_carriers
-        ,d_year as year
+    (select 
+ 	w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+ 	,'DIAMOND' || ',' || 'AIRBORNE' as ship_carriers
+       ,d_year as year
  	,sum(case when d_moy = 1 
- 		  then ws_sales_price* ws_quantity else 0 end) as jan_sales
- 		  ,sum(case when d_moy = 2 
- 		  	    then ws_sales_price* ws_quantity else 0 end) as feb_sales
- 			    ,sum(case when d_moy = 3 
- 			    	      then ws_sales_price* ws_quantity else 0 end) as mar_sales
- 				      ,sum(case when d_moy = 4 
- 				      		then ws_sales_price* ws_quantity else 0 end) as apr_sales
- 						,sum(case when d_moy = 5 
- 							  then ws_sales_price* ws_quantity else 0 end) as may_sales
- 							  ,sum(case when d_moy = 6 
- 							  	    then ws_sales_price* ws_quantity else 0 end) as jun_sales
- 								    ,sum(case when d_moy = 7 
- 								    	      then ws_sales_price* ws_quantity else 0 end) as jul_sales
- 									      ,sum(case when d_moy = 8 
- 									      		then ws_sales_price* ws_quantity else 0 end) as aug_sales
- 											,sum(case when d_moy = 9 
- 												  then ws_sales_price* ws_quantity else 0 end) as sep_sales
- 												  ,sum(case when d_moy = 10 
- 												  	    then ws_sales_price* ws_quantity else 0 end) as oct_sales
- 													    ,sum(case when d_moy = 11
- 													    	      then ws_sales_price* ws_quantity else 0 end) as nov_sales
- 														      ,sum(case when d_moy = 12
- 														      		then ws_sales_price* ws_quantity else 0 end) as dec_sales
- 																,sum(case when d_moy = 1 
- 																	  then ws_net_paid_inc_tax * ws_quantity else 0 end) as jan_net
- 																	  ,sum(case when d_moy = 2
- 																	  	    then ws_net_paid_inc_tax * ws_quantity else 0 end) as feb_net
- 																		    ,sum(case when d_moy = 3 
- 																		    	      then ws_net_paid_inc_tax * ws_quantity else 0 end) as mar_net
- 																			      ,sum(case when d_moy = 4 
- 																			      		then ws_net_paid_inc_tax * ws_quantity else 0 end) as apr_net
- 																					,sum(case when d_moy = 5 
- 																						  then ws_net_paid_inc_tax * ws_quantity else 0 end) as may_net
- 																						  ,sum(case when d_moy = 6 
- 																						  	    then ws_net_paid_inc_tax * ws_quantity else 0 end) as jun_net
- 																							    ,sum(case when d_moy = 7 
- 																							    	      then ws_net_paid_inc_tax * ws_quantity else 0 end) as jul_net
- 																								      ,sum(case when d_moy = 8 
- 																								      		then ws_net_paid_inc_tax * ws_quantity else 0 end) as aug_net
- 																										,sum(case when d_moy = 9 
- 																											  then ws_net_paid_inc_tax * ws_quantity else 0 end) as sep_net
- 																											  ,sum(case when d_moy = 10 
- 																											  	    then ws_net_paid_inc_tax * ws_quantity else 0 end) as oct_net
- 																												    ,sum(case when d_moy = 11
- 																												    	      then ws_net_paid_inc_tax * ws_quantity else 0 end) as nov_net
- 																													      ,sum(case when d_moy = 12
- 																													      		then ws_net_paid_inc_tax * ws_quantity else 0 end) as dec_net
+ 		then ws_sales_price* ws_quantity else 0 end) as jan_sales
+ 	,sum(case when d_moy = 2 
+ 		then ws_sales_price* ws_quantity else 0 end) as feb_sales
+ 	,sum(case when d_moy = 3 
+ 		then ws_sales_price* ws_quantity else 0 end) as mar_sales
+ 	,sum(case when d_moy = 4 
+ 		then ws_sales_price* ws_quantity else 0 end) as apr_sales
+ 	,sum(case when d_moy = 5 
+ 		then ws_sales_price* ws_quantity else 0 end) as may_sales
+ 	,sum(case when d_moy = 6 
+ 		then ws_sales_price* ws_quantity else 0 end) as jun_sales
+ 	,sum(case when d_moy = 7 
+ 		then ws_sales_price* ws_quantity else 0 end) as jul_sales
+ 	,sum(case when d_moy = 8 
+ 		then ws_sales_price* ws_quantity else 0 end) as aug_sales
+ 	,sum(case when d_moy = 9 
+ 		then ws_sales_price* ws_quantity else 0 end) as sep_sales
+ 	,sum(case when d_moy = 10 
+ 		then ws_sales_price* ws_quantity else 0 end) as oct_sales
+ 	,sum(case when d_moy = 11
+ 		then ws_sales_price* ws_quantity else 0 end) as nov_sales
+ 	,sum(case when d_moy = 12
+ 		then ws_sales_price* ws_quantity else 0 end) as dec_sales
+ 	,sum(case when d_moy = 1 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jan_net
+ 	,sum(case when d_moy = 2
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as feb_net
+ 	,sum(case when d_moy = 3 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as mar_net
+ 	,sum(case when d_moy = 4 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as apr_net
+ 	,sum(case when d_moy = 5 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as may_net
+ 	,sum(case when d_moy = 6 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jun_net
+ 	,sum(case when d_moy = 7 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jul_net
+ 	,sum(case when d_moy = 8 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as aug_net
+ 	,sum(case when d_moy = 9 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as sep_net
+ 	,sum(case when d_moy = 10 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as oct_net
+ 	,sum(case when d_moy = 11
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as nov_net
+ 	,sum(case when d_moy = 12
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as dec_net
      from
           web_sales
          ,warehouse
          ,date_dim
          ,time_dim
- 	   ,ship_mode
+ 	  ,ship_mode
      where
-            web_sales.ws_warehouse_sk =  warehouse.w_warehouse_sk
-        and web_sales.ws_sold_date_sk = date_dim.d_date_sk
-        and web_sales.ws_sold_time_sk = time_dim.t_time_sk
- 	and web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk
+            ws_warehouse_sk =  w_warehouse_sk
+        and ws_sold_date_sk = d_date_sk
+        and ws_sold_time_sk = t_time_sk
+ 	and ws_ship_mode_sk = sm_ship_mode_sk
         and d_year = 2002
  	and t_time between 49530 and 49530+28800 
  	and sm_carrier in ('DIAMOND','AIRBORNE')
@@ -343,75 +345,76 @@ select
  	,w_state
  	,w_country
        ,d_year
+ 	)
  union all
-    select 
-    w_warehouse_name
-    ,w_warehouse_sq_ft
-    ,w_city
-    ,w_county
-    ,w_state
-    ,w_country
-        ,concat('DIAMOND', ',', 'AIRBORNE') as ship_carriers
+    (select 
+ 	w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+ 	,'DIAMOND' || ',' || 'AIRBORNE' as ship_carriers
        ,d_year as year
-       ,sum(case when d_moy = 1 
-       		 then cs_ext_sales_price* cs_quantity else 0 end) as jan_sales
- 		 ,sum(case when d_moy = 2 
- 		 	   then cs_ext_sales_price* cs_quantity else 0 end) as feb_sales
- 			   ,sum(case when d_moy = 3 
- 			   	     then cs_ext_sales_price* cs_quantity else 0 end) as mar_sales
- 				     ,sum(case when d_moy = 4 
- 				     	       then cs_ext_sales_price* cs_quantity else 0 end) as apr_sales
- 					       ,sum(case when d_moy = 5 
- 					       		 then cs_ext_sales_price* cs_quantity else 0 end) as may_sales
- 							 ,sum(case when d_moy = 6 
- 							 	   then cs_ext_sales_price* cs_quantity else 0 end) as jun_sales
- 								   ,sum(case when d_moy = 7 
- 								   	     then cs_ext_sales_price* cs_quantity else 0 end) as jul_sales
- 									     ,sum(case when d_moy = 8 
- 									     	       then cs_ext_sales_price* cs_quantity else 0 end) as aug_sales
- 										       ,sum(case when d_moy = 9 
- 										       		 then cs_ext_sales_price* cs_quantity else 0 end) as sep_sales
- 												 ,sum(case when d_moy = 10 
- 												 	   then cs_ext_sales_price* cs_quantity else 0 end) as oct_sales
- 													   ,sum(case when d_moy = 11
- 													   	     then cs_ext_sales_price* cs_quantity else 0 end) as nov_sales
- 														     ,sum(case when d_moy = 12
- 														     	       then cs_ext_sales_price* cs_quantity else 0 end) as dec_sales
- 															       ,sum(case when d_moy = 1 
- 															       		 then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jan_net
- 																	 ,sum(case when d_moy = 2 
- 																	 	   then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as feb_net
- 																		   ,sum(case when d_moy = 3 
- 																		   	     then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as mar_net
- 																			     ,sum(case when d_moy = 4 
- 																			     	       then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as apr_net
- 																				       ,sum(case when d_moy = 5 
- 																				       		 then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as may_net
- 																						 ,sum(case when d_moy = 6 
- 																						 	   then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jun_net
- 																							   ,sum(case when d_moy = 7 
- 																							   	     then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jul_net
- 																								     ,sum(case when d_moy = 8 
- 																								     	       then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as aug_net
- 																									       ,sum(case when d_moy = 9 
- 																									       		 then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as sep_net
- 																											 ,sum(case when d_moy = 10 
- 																											 	   then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as oct_net
- 																												   ,sum(case when d_moy = 11
- 																												   	     then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as nov_net
- 																													     ,sum(case when d_moy = 12
- 																													     	       then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as dec_net
+ 	,sum(case when d_moy = 1 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jan_sales
+ 	,sum(case when d_moy = 2 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as feb_sales
+ 	,sum(case when d_moy = 3 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as mar_sales
+ 	,sum(case when d_moy = 4 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as apr_sales
+ 	,sum(case when d_moy = 5 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as may_sales
+ 	,sum(case when d_moy = 6 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jun_sales
+ 	,sum(case when d_moy = 7 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jul_sales
+ 	,sum(case when d_moy = 8 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as aug_sales
+ 	,sum(case when d_moy = 9 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as sep_sales
+ 	,sum(case when d_moy = 10 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as oct_sales
+ 	,sum(case when d_moy = 11
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as nov_sales
+ 	,sum(case when d_moy = 12
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as dec_sales
+ 	,sum(case when d_moy = 1 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jan_net
+ 	,sum(case when d_moy = 2 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as feb_net
+ 	,sum(case when d_moy = 3 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as mar_net
+ 	,sum(case when d_moy = 4 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as apr_net
+ 	,sum(case when d_moy = 5 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as may_net
+ 	,sum(case when d_moy = 6 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jun_net
+ 	,sum(case when d_moy = 7 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jul_net
+ 	,sum(case when d_moy = 8 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as aug_net
+ 	,sum(case when d_moy = 9 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as sep_net
+ 	,sum(case when d_moy = 10 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as oct_net
+ 	,sum(case when d_moy = 11
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as nov_net
+ 	,sum(case when d_moy = 12
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as dec_net
      from
           catalog_sales
          ,warehouse
          ,date_dim
          ,time_dim
- 	  ,ship_mode
+ 	 ,ship_mode
      where
-            catalog_sales.cs_warehouse_sk =  warehouse.w_warehouse_sk
-        and catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
-        and catalog_sales.cs_sold_time_sk = time_dim.t_time_sk
- 	and catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk
+            cs_warehouse_sk =  w_warehouse_sk
+        and cs_sold_date_sk = d_date_sk
+        and cs_sold_time_sk = t_time_sk
+ 	and cs_ship_mode_sk = sm_ship_mode_sk
         and d_year = 2002
  	and t_time between 49530 AND 49530+28800 
  	and sm_carrier in ('DIAMOND','AIRBORNE')
@@ -423,6 +426,7 @@ select
  	,w_state
  	,w_country
        ,d_year
+     ) 
  ) x
  group by 
         w_warehouse_name

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query68.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query68.q.out b/ql/src/test/results/clientpositive/perf/query68.q.out
index 84f701b..bd9b5ec 100644
--- a/ql/src/test/results/clientpositive/perf/query68.q.out
+++ b/ql/src/test/results/clientpositive/perf/query68.q.out
@@ -1,6 +1,84 @@
-PREHOOK: query: explain select c_last_name ,c_first_name ,ca_city ,bought_city ,ss_ticket_number ,extended_price ,extended_tax ,list_price from (select ss_ticket_number ,ss_customer_sk ,ca_city bought_city ,sum(ss_ext_sales_price) extended_price ,sum(ss_ext_list_price) list_price ,sum(ss_ext_tax) extended_tax from store_sales ,date_dim ,store ,household_demographics ,customer_address where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and store_sales.ss_addr_sk = customer_address.ca_address_sk and date_dim.d_dom between 1 and 2 and (household_demographics.hd_dep_count = 4 or household_demographics.hd_vehicle_count= 2) and date_dim.d_year in (1998,1998+1,1998+2) and store.s_city in ('Rosedale','Bethlehem') group by ss_ticket_number ,ss_customer_sk ,ss_addr_sk,ca_city) dn ,customer ,customer_address current_addr where dn.ss_customer_sk = customer.c_customer_sk and customer
 .c_current_addr_sk = current_addr.ca_address_sk and current_addr.ca_city <> bought_city order by c_last_name ,ss_ticket_number limit 100
+PREHOOK: query: explain
+select  c_last_name
+       ,c_first_name
+       ,ca_city
+       ,bought_city
+       ,ss_ticket_number
+       ,extended_price
+       ,extended_tax
+       ,list_price
+ from (select ss_ticket_number
+             ,ss_customer_sk
+             ,ca_city bought_city
+             ,sum(ss_ext_sales_price) extended_price 
+             ,sum(ss_ext_list_price) list_price
+             ,sum(ss_ext_tax) extended_tax 
+       from store_sales
+           ,date_dim
+           ,store
+           ,household_demographics
+           ,customer_address 
+       where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+         and store_sales.ss_store_sk = store.s_store_sk  
+        and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+        and store_sales.ss_addr_sk = customer_address.ca_address_sk
+        and date_dim.d_dom between 1 and 2 
+        and (household_demographics.hd_dep_count = 2 or
+             household_demographics.hd_vehicle_count= 1)
+        and date_dim.d_year in (1998,1998+1,1998+2)
+        and store.s_city in ('Cedar Grove','Wildwood')
+       group by ss_ticket_number
+               ,ss_customer_sk
+               ,ss_addr_sk,ca_city) dn
+      ,customer
+      ,customer_address current_addr
+ where ss_customer_sk = c_customer_sk
+   and customer.c_current_addr_sk = current_addr.ca_address_sk
+   and current_addr.ca_city <> bought_city
+ order by c_last_name
+         ,ss_ticket_number
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select c_last_name ,c_first_name ,ca_city ,bought_city ,ss_ticket_number ,extended_price ,extended_tax ,list_price from (select ss_ticket_number ,ss_customer_sk ,ca_city bought_city ,sum(ss_ext_sales_price) extended_price ,sum(ss_ext_list_price) list_price ,sum(ss_ext_tax) extended_tax from store_sales ,date_dim ,store ,household_demographics ,customer_address where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and store_sales.ss_addr_sk = customer_address.ca_address_sk and date_dim.d_dom between 1 and 2 and (household_demographics.hd_dep_count = 4 or household_demographics.hd_vehicle_count= 2) and date_dim.d_year in (1998,1998+1,1998+2) and store.s_city in ('Rosedale','Bethlehem') group by ss_ticket_number ,ss_customer_sk ,ss_addr_sk,ca_city) dn ,customer ,customer_address current_addr where dn.ss_customer_sk = customer.c_customer_sk and custome
 r.c_current_addr_sk = current_addr.ca_address_sk and current_addr.ca_city <> bought_city order by c_last_name ,ss_ticket_number limit 100
+POSTHOOK: query: explain
+select  c_last_name
+       ,c_first_name
+       ,ca_city
+       ,bought_city
+       ,ss_ticket_number
+       ,extended_price
+       ,extended_tax
+       ,list_price
+ from (select ss_ticket_number
+             ,ss_customer_sk
+             ,ca_city bought_city
+             ,sum(ss_ext_sales_price) extended_price 
+             ,sum(ss_ext_list_price) list_price
+             ,sum(ss_ext_tax) extended_tax 
+       from store_sales
+           ,date_dim
+           ,store
+           ,household_demographics
+           ,customer_address 
+       where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+         and store_sales.ss_store_sk = store.s_store_sk  
+        and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+        and store_sales.ss_addr_sk = customer_address.ca_address_sk
+        and date_dim.d_dom between 1 and 2 
+        and (household_demographics.hd_dep_count = 2 or
+             household_demographics.hd_vehicle_count= 1)
+        and date_dim.d_year in (1998,1998+1,1998+2)
+        and store.s_city in ('Cedar Grove','Wildwood')
+       group by ss_ticket_number
+               ,ss_customer_sk
+               ,ss_addr_sk,ca_city) dn
+      ,customer
+      ,customer_address current_addr
+ where ss_customer_sk = c_customer_sk
+   and customer.c_current_addr_sk = current_addr.ca_address_sk
+   and current_addr.ca_city <> bought_city
+ order by c_last_name
+         ,ss_ticket_number
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -88,7 +166,7 @@ Stage-0
                                       Select Operator [SEL_17] (rows=7200 width=107)
                                         Output:["_col0"]
                                         Filter Operator [FIL_79] (rows=7200 width=107)
-                                          predicate:(((hd_dep_count = 4) or (hd_vehicle_count = 2)) and hd_demo_sk is not null)
+                                          predicate:(((hd_dep_count = 2) or (hd_vehicle_count = 1)) and hd_demo_sk is not null)
                                           TableScan [TS_15] (rows=7200 width=107)
                                             default@household_demographics,household_demographics,Tbl:COMPLETE,Col:NONE,Output:["hd_demo_sk","hd_dep_count","hd_vehicle_count"]
                                   <-Reducer 10 [SIMPLE_EDGE]
@@ -102,7 +180,7 @@ Stage-0
                                           Select Operator [SEL_14] (rows=852 width=1910)
                                             Output:["_col0"]
                                             Filter Operator [FIL_78] (rows=852 width=1910)
-                                              predicate:((s_city) IN ('Rosedale', 'Bethlehem') and s_store_sk is not null)
+                                              predicate:((s_city) IN ('Cedar Grove', 'Wildwood') and s_store_sk is not null)
                                               TableScan [TS_12] (rows=1704 width=1910)
                                                 default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_city"]
                                       <-Reducer 9 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query69.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query69.q.out b/ql/src/test/results/clientpositive/perf/query69.q.out
index 7ee80a6..a55c368 100644
--- a/ql/src/test/results/clientpositive/perf/query69.q.out
+++ b/ql/src/test/results/clientpositive/perf/query69.q.out
@@ -1,4 +1,5 @@
-PREHOOK: query: explain select  
+PREHOOK: query: explain
+select  
   cd_gender,
   cd_marital_status,
   cd_education_status,
@@ -43,7 +44,8 @@ PREHOOK: query: explain select
           cd_credit_rating
  limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select  
+POSTHOOK: query: explain
+select  
   cd_gender,
   cd_marital_status,
   cd_education_status,

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query7.q.out b/ql/src/test/results/clientpositive/perf/query7.q.out
index af77d9a..d3045d1 100644
--- a/ql/src/test/results/clientpositive/perf/query7.q.out
+++ b/ql/src/test/results/clientpositive/perf/query7.q.out
@@ -1,6 +1,42 @@
-PREHOOK: query: explain select i_item_id, avg(ss_quantity) agg1, avg(ss_list_price) agg2, avg(ss_coupon_amt) agg3, avg(ss_sales_price) agg4 from store_sales, customer_demographics, date_dim, item, promotion where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_item_sk = item.i_item_sk and store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk and store_sales.ss_promo_sk = promotion.p_promo_sk and cd_gender = 'F' and cd_marital_status = 'W' and cd_education_status = 'Primary' and (p_channel_email = 'N' or p_channel_event = 'N') and d_year = 1998 group by i_item_id order by i_item_id limit 100
+PREHOOK: query: explain
+select  i_item_id, 
+        avg(ss_quantity) agg1,
+        avg(ss_list_price) agg2,
+        avg(ss_coupon_amt) agg3,
+        avg(ss_sales_price) agg4 
+ from store_sales, customer_demographics, date_dim, item, promotion
+ where ss_sold_date_sk = d_date_sk and
+       ss_item_sk = i_item_sk and
+       ss_cdemo_sk = cd_demo_sk and
+       ss_promo_sk = p_promo_sk and
+       cd_gender = 'F' and 
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       (p_channel_email = 'N' or p_channel_event = 'N') and
+       d_year = 1998 
+ group by i_item_id
+ order by i_item_id
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select i_item_id, avg(ss_quantity) agg1, avg(ss_list_price) agg2, avg(ss_coupon_amt) agg3, avg(ss_sales_price) agg4 from store_sales, customer_demographics, date_dim, item, promotion where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_item_sk = item.i_item_sk and store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk and store_sales.ss_promo_sk = promotion.p_promo_sk and cd_gender = 'F' and cd_marital_status = 'W' and cd_education_status = 'Primary' and (p_channel_email = 'N' or p_channel_event = 'N') and d_year = 1998 group by i_item_id order by i_item_id limit 100
+POSTHOOK: query: explain
+select  i_item_id, 
+        avg(ss_quantity) agg1,
+        avg(ss_list_price) agg2,
+        avg(ss_coupon_amt) agg3,
+        avg(ss_sales_price) agg4 
+ from store_sales, customer_demographics, date_dim, item, promotion
+ where ss_sold_date_sk = d_date_sk and
+       ss_item_sk = i_item_sk and
+       ss_cdemo_sk = cd_demo_sk and
+       ss_promo_sk = p_promo_sk and
+       cd_gender = 'F' and 
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       (p_channel_email = 'N' or p_channel_event = 'N') and
+       d_year = 1998 
+ group by i_item_id
+ order by i_item_id
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 


[17/17] hive git commit: HIVE-14671 : Merge branch 'master' into hive-14535 (Wei Zheng)

Posted by we...@apache.org.
HIVE-14671 : Merge branch 'master' into hive-14535 (Wei Zheng)


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

Branch: refs/heads/hive-14535
Commit: 3302150438ff960a72dabe0533b7e0d008ed0ec1
Parents: 7ab6f01 86b1877
Author: Wei Zheng <we...@apache.org>
Authored: Tue May 30 17:09:36 2017 -0700
Committer: Wei Zheng <we...@apache.org>
Committed: Tue May 30 17:09:36 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/Utilities.java   |  48 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   2 +-
 .../hadoop/hive/ql/exec/TestUtilities.java      | 104 ++-
 .../test/queries/clientpositive/perf/query1.q   |   7 +-
 .../test/queries/clientpositive/perf/query10.q  |  61 ++
 .../test/queries/clientpositive/perf/query12.q  |  46 +-
 .../test/queries/clientpositive/perf/query13.q  | 106 +--
 .../test/queries/clientpositive/perf/query14.q  |   5 +-
 .../test/queries/clientpositive/perf/query15.q  |  23 +-
 .../test/queries/clientpositive/perf/query16.q  |   9 +-
 .../test/queries/clientpositive/perf/query17.q  |  48 +-
 .../test/queries/clientpositive/perf/query18.q  |  37 +-
 .../test/queries/clientpositive/perf/query19.q  |  28 +-
 .../test/queries/clientpositive/perf/query2.q   |  62 ++
 .../test/queries/clientpositive/perf/query20.q  |  32 +-
 .../test/queries/clientpositive/perf/query21.q  |   3 +
 .../test/queries/clientpositive/perf/query22.q  |   3 +
 .../test/queries/clientpositive/perf/query23.q  |   6 +-
 .../test/queries/clientpositive/perf/query24.q  | 100 +--
 .../test/queries/clientpositive/perf/query25.q  |  51 +-
 .../test/queries/clientpositive/perf/query26.q  |  24 +-
 .../test/queries/clientpositive/perf/query27.q  |   3 +
 .../test/queries/clientpositive/perf/query28.q  |   5 +-
 .../test/queries/clientpositive/perf/query29.q  |  50 +-
 .../test/queries/clientpositive/perf/query3.q   |  24 +-
 .../test/queries/clientpositive/perf/query30.q  |   7 +-
 .../test/queries/clientpositive/perf/query31.q  |  54 +-
 .../test/queries/clientpositive/perf/query32.q  |  47 +-
 .../test/queries/clientpositive/perf/query33.q  |   6 +-
 .../test/queries/clientpositive/perf/query34.q  |  33 +-
 .../test/queries/clientpositive/perf/query35.q  |  59 ++
 .../test/queries/clientpositive/perf/query36.q  |   9 +-
 .../test/queries/clientpositive/perf/query37.q  |   3 +
 .../test/queries/clientpositive/perf/query38.q  |   3 +
 .../test/queries/clientpositive/perf/query39.q  |  56 +-
 .../test/queries/clientpositive/perf/query40.q  |   3 +
 .../test/queries/clientpositive/perf/query42.q  |  25 +-
 .../test/queries/clientpositive/perf/query43.q  |  22 +-
 .../test/queries/clientpositive/perf/query44.q  |  37 +
 .../test/queries/clientpositive/perf/query45.q  |  22 +
 .../test/queries/clientpositive/perf/query46.q  |  38 +-
 .../test/queries/clientpositive/perf/query48.q  |  69 +-
 .../test/queries/clientpositive/perf/query49.q  | 129 +++
 .../test/queries/clientpositive/perf/query5.q   |   3 +
 .../test/queries/clientpositive/perf/query50.q  |  25 +-
 .../test/queries/clientpositive/perf/query51.q  |  14 +-
 .../test/queries/clientpositive/perf/query52.q  |  25 +-
 .../test/queries/clientpositive/perf/query53.q  |  30 +
 .../test/queries/clientpositive/perf/query54.q  |  59 +-
 .../test/queries/clientpositive/perf/query55.q  |  17 +-
 .../test/queries/clientpositive/perf/query56.q  |   7 +-
 .../test/queries/clientpositive/perf/query58.q  |  86 +-
 .../test/queries/clientpositive/perf/query59.q  |  46 +
 .../test/queries/clientpositive/perf/query6.q   |   7 +-
 .../test/queries/clientpositive/perf/query60.q  |   5 +-
 .../test/queries/clientpositive/perf/query61.q  |  46 +
 .../test/queries/clientpositive/perf/query63.q  |  31 +
 .../test/queries/clientpositive/perf/query64.q  | 121 ++-
 .../test/queries/clientpositive/perf/query65.q  |  65 +-
 .../test/queries/clientpositive/perf/query66.q  | 259 +++---
 .../test/queries/clientpositive/perf/query67.q  |   3 +-
 .../test/queries/clientpositive/perf/query68.q  |  45 +-
 .../test/queries/clientpositive/perf/query69.q  |   7 +-
 .../test/queries/clientpositive/perf/query7.q   |  24 +-
 .../test/queries/clientpositive/perf/query70.q  |  19 +-
 .../test/queries/clientpositive/perf/query71.q  |  42 +-
 .../test/queries/clientpositive/perf/query72.q  |   3 +
 .../test/queries/clientpositive/perf/query73.q  |  30 +-
 .../test/queries/clientpositive/perf/query75.q  |  73 +-
 .../test/queries/clientpositive/perf/query76.q  |  27 +-
 .../test/queries/clientpositive/perf/query77.q  | 109 +++
 .../test/queries/clientpositive/perf/query78.q  |  60 ++
 .../test/queries/clientpositive/perf/query79.q  |  26 +-
 .../test/queries/clientpositive/perf/query8.q   |   3 +
 .../test/queries/clientpositive/perf/query80.q  |   3 +
 .../test/queries/clientpositive/perf/query81.q  |   7 +-
 .../test/queries/clientpositive/perf/query82.q  |   3 +
 .../test/queries/clientpositive/perf/query83.q  |   6 +-
 .../test/queries/clientpositive/perf/query84.q  |  24 +-
 .../test/queries/clientpositive/perf/query85.q  |  87 +-
 .../test/queries/clientpositive/perf/query86.q  |   9 +-
 .../test/queries/clientpositive/perf/query87.q  |   3 +
 .../test/queries/clientpositive/perf/query88.q  |  54 +-
 .../test/queries/clientpositive/perf/query89.q  |   9 +-
 .../test/queries/clientpositive/perf/query9.q   |  19 +-
 .../test/queries/clientpositive/perf/query90.q  |  24 +-
 .../test/queries/clientpositive/perf/query91.q  |  33 +-
 .../test/queries/clientpositive/perf/query92.q  |  33 +-
 .../test/queries/clientpositive/perf/query93.q  |  21 +-
 .../test/queries/clientpositive/perf/query94.q  |  32 +-
 .../test/queries/clientpositive/perf/query95.q  |  35 +-
 .../test/queries/clientpositive/perf/query96.q  |  19 +-
 .../test/queries/clientpositive/perf/query97.q  |  28 +-
 .../test/queries/clientpositive/perf/query98.q  |  39 +-
 .../test/queries/clientpositive/perf/query99.q  |  37 +
 .../results/clientpositive/perf/query1.q.out    |   6 +-
 .../results/clientpositive/perf/query10.q.out   | 296 +++++++
 .../results/clientpositive/perf/query12.q.out   | 178 ++--
 .../results/clientpositive/perf/query13.q.out   | 204 +++--
 .../results/clientpositive/perf/query15.q.out   |  38 +-
 .../results/clientpositive/perf/query16.q.out   | 302 +++----
 .../results/clientpositive/perf/query17.q.out   |  88 +-
 .../results/clientpositive/perf/query18.q.out   |  70 +-
 .../results/clientpositive/perf/query19.q.out   |  48 +-
 .../results/clientpositive/perf/query2.q.out    | 259 ++++++
 .../results/clientpositive/perf/query20.q.out   |  74 +-
 .../results/clientpositive/perf/query23.q.out   |   8 +-
 .../results/clientpositive/perf/query24.q.out   | 190 ++--
 .../results/clientpositive/perf/query25.q.out   | 100 ++-
 .../results/clientpositive/perf/query26.q.out   |  40 +-
 .../results/clientpositive/perf/query28.q.out   |   6 +-
 .../results/clientpositive/perf/query29.q.out   |  98 ++-
 .../results/clientpositive/perf/query3.q.out    |  40 +-
 .../results/clientpositive/perf/query30.q.out   |   6 +-
 .../results/clientpositive/perf/query31.q.out   | 585 +++++++------
 .../results/clientpositive/perf/query32.q.out   | 248 +++---
 .../results/clientpositive/perf/query33.q.out   |   6 +-
 .../results/clientpositive/perf/query34.q.out   |  66 +-
 .../results/clientpositive/perf/query35.q.out   | 292 +++++++
 .../results/clientpositive/perf/query36.q.out   |  12 +-
 .../results/clientpositive/perf/query39.q.out   | 108 ++-
 .../results/clientpositive/perf/query42.q.out   | 126 ++-
 .../results/clientpositive/perf/query43.q.out   |  36 +-
 .../results/clientpositive/perf/query44.q.out   | 279 ++++++
 .../results/clientpositive/perf/query45.q.out   | 180 ++++
 .../results/clientpositive/perf/query46.q.out   |  72 +-
 .../results/clientpositive/perf/query48.q.out   | 130 ++-
 .../results/clientpositive/perf/query49.q.out   | 504 +++++++++++
 .../results/clientpositive/perf/query50.q.out   |  44 +-
 .../results/clientpositive/perf/query51.q.out   |  26 +-
 .../results/clientpositive/perf/query52.q.out   |  42 +-
 .../results/clientpositive/perf/query53.q.out   | 141 +++
 .../results/clientpositive/perf/query54.q.out   | 521 +++++++----
 .../results/clientpositive/perf/query55.q.out   |  26 +-
 .../results/clientpositive/perf/query56.q.out   |   6 +-
 .../results/clientpositive/perf/query58.q.out   | 674 ++++++++------
 .../results/clientpositive/perf/query59.q.out   | 238 +++++
 .../results/clientpositive/perf/query6.q.out    |   6 +-
 .../results/clientpositive/perf/query60.q.out   |   6 +-
 .../results/clientpositive/perf/query61.q.out   | 300 +++++++
 .../results/clientpositive/perf/query62.q.out   | 164 ++++
 .../results/clientpositive/perf/query63.q.out   | 143 +++
 .../results/clientpositive/perf/query64.q.out   | 244 +++++-
 .../results/clientpositive/perf/query65.q.out   | 124 ++-
 .../results/clientpositive/perf/query66.q.out   | 512 +++++------
 .../results/clientpositive/perf/query68.q.out   |  86 +-
 .../results/clientpositive/perf/query69.q.out   |   6 +-
 .../results/clientpositive/perf/query7.q.out    |  40 +-
 .../results/clientpositive/perf/query70.q.out   |  34 +-
 .../results/clientpositive/perf/query71.q.out   |  76 +-
 .../results/clientpositive/perf/query73.q.out   |  60 +-
 .../results/clientpositive/perf/query75.q.out   | 874 +++++++++++--------
 .../results/clientpositive/perf/query76.q.out   |  46 +-
 .../results/clientpositive/perf/query77.q.out   | 509 +++++++++++
 .../results/clientpositive/perf/query78.q.out   | 302 +++++++
 .../results/clientpositive/perf/query79.q.out   |  68 +-
 .../results/clientpositive/perf/query81.q.out   |   6 +-
 .../results/clientpositive/perf/query83.q.out   |   6 +-
 .../results/clientpositive/perf/query84.q.out   |  66 +-
 .../results/clientpositive/perf/query85.q.out   | 168 +++-
 .../results/clientpositive/perf/query86.q.out   |  12 +-
 .../results/clientpositive/perf/query88.q.out   |  96 +-
 .../results/clientpositive/perf/query89.q.out   |  12 +-
 .../results/clientpositive/perf/query9.q.out    |  26 +-
 .../results/clientpositive/perf/query90.q.out   |  42 +-
 .../results/clientpositive/perf/query91.q.out   |  62 +-
 .../results/clientpositive/perf/query92.q.out   | 229 +++--
 .../results/clientpositive/perf/query93.q.out   |  34 +-
 .../results/clientpositive/perf/query94.q.out   | 333 ++++---
 .../results/clientpositive/perf/query95.q.out   | 341 +++++---
 .../results/clientpositive/perf/query96.q.out   | 144 +--
 .../results/clientpositive/perf/query97.q.out   |  52 +-
 .../results/clientpositive/perf/query98.q.out   |  86 +-
 .../results/clientpositive/perf/query99.q.out   | 164 ++++
 174 files changed, 12143 insertions(+), 3100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/33021504/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------


[04/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query77.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query77.q.out b/ql/src/test/results/clientpositive/perf/query77.q.out
new file mode 100644
index 0000000..d46ba6b
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query77.q.out
@@ -0,0 +1,509 @@
+Warning: Shuffle Join MERGEJOIN[189][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 15' is a cross product
+PREHOOK: query: explain
+with ss as
+ (select s_store_sk,
+         sum(ss_ext_sales_price) as sales,
+         sum(ss_net_profit) as profit
+ from store_sales,
+      date_dim,
+      store
+ where ss_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date) 
+                  and (cast('1998-08-04' as date) +  30 days) 
+       and ss_store_sk = s_store_sk
+ group by s_store_sk)
+ ,
+ sr as
+ (select s_store_sk,
+         sum(sr_return_amt) as returns,
+         sum(sr_net_loss) as profit_loss
+ from store_returns,
+      date_dim,
+      store
+ where sr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and sr_store_sk = s_store_sk
+ group by s_store_sk), 
+ cs as
+ (select cs_call_center_sk,
+        sum(cs_ext_sales_price) as sales,
+        sum(cs_net_profit) as profit
+ from catalog_sales,
+      date_dim
+ where cs_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+ group by cs_call_center_sk 
+ ), 
+ cr as
+ (select
+        sum(cr_return_amount) as returns,
+        sum(cr_net_loss) as profit_loss
+ from catalog_returns,
+      date_dim
+ where cr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+ ), 
+ ws as
+ ( select wp_web_page_sk,
+        sum(ws_ext_sales_price) as sales,
+        sum(ws_net_profit) as profit
+ from web_sales,
+      date_dim,
+      web_page
+ where ws_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and ws_web_page_sk = wp_web_page_sk
+ group by wp_web_page_sk), 
+ wr as
+ (select wp_web_page_sk,
+        sum(wr_return_amt) as returns,
+        sum(wr_net_loss) as profit_loss
+ from web_returns,
+      date_dim,
+      web_page
+ where wr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and wr_web_page_sk = wp_web_page_sk
+ group by wp_web_page_sk)
+  select  channel
+        , id
+        , sum(sales) as sales
+        , sum(returns) as returns
+        , sum(profit) as profit
+ from 
+ (select 'store channel' as channel
+        , ss.s_store_sk as id
+        , sales
+        , coalesce(returns, 0) as returns
+        , (profit - coalesce(profit_loss,0)) as profit
+ from   ss left join sr
+        on  ss.s_store_sk = sr.s_store_sk
+ union all
+ select 'catalog channel' as channel
+        , cs_call_center_sk as id
+        , sales
+        , returns
+        , (profit - profit_loss) as profit
+ from  cs
+       , cr
+ union all
+ select 'web channel' as channel
+        , ws.wp_web_page_sk as id
+        , sales
+        , coalesce(returns, 0) returns
+        , (profit - coalesce(profit_loss,0)) as profit
+ from   ws left join wr
+        on  ws.wp_web_page_sk = wr.wp_web_page_sk
+ ) x
+ group by rollup (channel, id)
+ order by channel
+         ,id
+ limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+with ss as
+ (select s_store_sk,
+         sum(ss_ext_sales_price) as sales,
+         sum(ss_net_profit) as profit
+ from store_sales,
+      date_dim,
+      store
+ where ss_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date) 
+                  and (cast('1998-08-04' as date) +  30 days) 
+       and ss_store_sk = s_store_sk
+ group by s_store_sk)
+ ,
+ sr as
+ (select s_store_sk,
+         sum(sr_return_amt) as returns,
+         sum(sr_net_loss) as profit_loss
+ from store_returns,
+      date_dim,
+      store
+ where sr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and sr_store_sk = s_store_sk
+ group by s_store_sk), 
+ cs as
+ (select cs_call_center_sk,
+        sum(cs_ext_sales_price) as sales,
+        sum(cs_net_profit) as profit
+ from catalog_sales,
+      date_dim
+ where cs_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+ group by cs_call_center_sk 
+ ), 
+ cr as
+ (select
+        sum(cr_return_amount) as returns,
+        sum(cr_net_loss) as profit_loss
+ from catalog_returns,
+      date_dim
+ where cr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+ ), 
+ ws as
+ ( select wp_web_page_sk,
+        sum(ws_ext_sales_price) as sales,
+        sum(ws_net_profit) as profit
+ from web_sales,
+      date_dim,
+      web_page
+ where ws_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and ws_web_page_sk = wp_web_page_sk
+ group by wp_web_page_sk), 
+ wr as
+ (select wp_web_page_sk,
+        sum(wr_return_amt) as returns,
+        sum(wr_net_loss) as profit_loss
+ from web_returns,
+      date_dim,
+      web_page
+ where wr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and wr_web_page_sk = wp_web_page_sk
+ group by wp_web_page_sk)
+  select  channel
+        , id
+        , sum(sales) as sales
+        , sum(returns) as returns
+        , sum(profit) as profit
+ from 
+ (select 'store channel' as channel
+        , ss.s_store_sk as id
+        , sales
+        , coalesce(returns, 0) as returns
+        , (profit - coalesce(profit_loss,0)) as profit
+ from   ss left join sr
+        on  ss.s_store_sk = sr.s_store_sk
+ union all
+ select 'catalog channel' as channel
+        , cs_call_center_sk as id
+        , sales
+        , returns
+        , (profit - profit_loss) as profit
+ from  cs
+       , cr
+ union all
+ select 'web channel' as channel
+        , ws.wp_web_page_sk as id
+        , sales
+        , coalesce(returns, 0) returns
+        , (profit - coalesce(profit_loss,0)) as profit
+ from   ws left join wr
+        on  ws.wp_web_page_sk = wr.wp_web_page_sk
+ ) x
+ group by rollup (channel, id)
+ order by channel
+         ,id
+ limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 10 <- Map 26 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 11 <- Map 25 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE)
+Reducer 12 <- Reducer 11 (SIMPLE_EDGE)
+Reducer 13 <- Map 27 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 14 <- Reducer 13 (SIMPLE_EDGE)
+Reducer 15 <- Reducer 14 (CUSTOM_SIMPLE_EDGE), Reducer 17 (CUSTOM_SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 16 <- Map 28 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 17 <- Reducer 16 (CUSTOM_SIMPLE_EDGE)
+Reducer 18 <- Map 29 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 19 <- Map 30 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 20 <- Reducer 19 (SIMPLE_EDGE)
+Reducer 21 <- Reducer 20 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 22 <- Map 31 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 23 <- Map 30 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE)
+Reducer 24 <- Reducer 23 (SIMPLE_EDGE)
+Reducer 3 <- Map 25 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 12 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 7 <- Union 6 (SIMPLE_EDGE)
+Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 8
+      File Output Operator [FS_131]
+        Limit [LIM_130] (rows=100 width=163)
+          Number of rows:100
+          Select Operator [SEL_129] (rows=956329968 width=163)
+            Output:["_col0","_col1","_col2","_col3","_col4"]
+          <-Reducer 7 [SIMPLE_EDGE]
+            SHUFFLE [RS_128]
+              Select Operator [SEL_127] (rows=956329968 width=163)
+                Output:["_col0","_col1","_col2","_col3","_col4"]
+                Group By Operator [GBY_126] (rows=956329968 width=163)
+                  Output:["_col0","_col1","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"],keys:KEY._col0, KEY._col1, KEY._col2
+                <-Union 6 [SIMPLE_EDGE]
+                  <-Reducer 15 [CONTAINS]
+                    Reduce Output Operator [RS_125]
+                      PartitionCols:_col0, _col1, _col2
+                      Group By Operator [GBY_124] (rows=1912659936 width=163)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col2)","sum(_col3)","sum(_col4)"],keys:_col0, _col1, 0
+                        Select Operator [SEL_75] (rows=158394413 width=360)
+                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                          Merge Join Operator [MERGEJOIN_189] (rows=158394413 width=360)
+                            Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4"]
+                          <-Reducer 14 [CUSTOM_SIMPLE_EDGE]
+                            PARTITION_ONLY_SHUFFLE [RS_72]
+                              Group By Operator [GBY_56] (rows=158394413 width=135)
+                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
+                              <-Reducer 13 [SIMPLE_EDGE]
+                                SHUFFLE [RS_55]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_54] (rows=316788826 width=135)
+                                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","sum(_col3)"],keys:_col1
+                                    Merge Join Operator [MERGEJOIN_182] (rows=316788826 width=135)
+                                      Conds:RS_50._col0=RS_51._col0(Inner),Output:["_col1","_col2","_col3"]
+                                    <-Map 9 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_51]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_49] (rows=8116 width=1119)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_169] (rows=8116 width=1119)
+                                            predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1998-08-04 00:00:00.0 AND 1998-09-03 00:00:00.0 and d_date_sk is not null)
+                                            TableScan [TS_3] (rows=73049 width=1119)
+                                              default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
+                                    <-Map 27 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_50]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_46] (rows=287989836 width=135)
+                                          Output:["_col0","_col1","_col2","_col3"]
+                                          Filter Operator [FIL_168] (rows=287989836 width=135)
+                                            predicate:cs_sold_date_sk is not null
+                                            TableScan [TS_44] (rows=287989836 width=135)
+                                              default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_call_center_sk","cs_ext_sales_price","cs_net_profit"]
+                          <-Reducer 17 [CUSTOM_SIMPLE_EDGE]
+                            PARTITION_ONLY_SHUFFLE [RS_73]
+                              Group By Operator [GBY_70] (rows=1 width=224)
+                                Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
+                              <-Reducer 16 [CUSTOM_SIMPLE_EDGE]
+                                PARTITION_ONLY_SHUFFLE [RS_69]
+                                  Group By Operator [GBY_68] (rows=1 width=224)
+                                    Output:["_col0","_col1"],aggregations:["sum(_col1)","sum(_col2)"]
+                                    Merge Join Operator [MERGEJOIN_183] (rows=31678769 width=106)
+                                      Conds:RS_64._col0=RS_65._col0(Inner),Output:["_col1","_col2"]
+                                    <-Map 9 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_65]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_63] (rows=8116 width=1119)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_171] (rows=8116 width=1119)
+                                            predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1998-08-04 00:00:00.0 AND 1998-09-03 00:00:00.0 and d_date_sk is not null)
+                                             Please refer to the previous TableScan [TS_3]
+                                    <-Map 28 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_64]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_60] (rows=28798881 width=106)
+                                          Output:["_col0","_col1","_col2"]
+                                          Filter Operator [FIL_170] (rows=28798881 width=106)
+                                            predicate:cr_returned_date_sk is not null
+                                            TableScan [TS_58] (rows=28798881 width=106)
+                                              default@catalog_returns,catalog_returns,Tbl:COMPLETE,Col:NONE,Output:["cr_returned_date_sk","cr_return_amount","cr_net_loss"]
+                  <-Reducer 21 [CONTAINS]
+                    Reduce Output Operator [RS_125]
+                      PartitionCols:_col0, _col1, _col2
+                      Group By Operator [GBY_124] (rows=1912659936 width=163)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col2)","sum(_col3)","sum(_col4)"],keys:_col0, _col1, 0
+                        Select Operator [SEL_121] (rows=95833780 width=135)
+                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                          Merge Join Operator [MERGEJOIN_190] (rows=95833780 width=135)
+                            Conds:RS_118._col0=RS_119._col0(Left Outer),Output:["_col0","_col1","_col2","_col4","_col5"]
+                          <-Reducer 20 [SIMPLE_EDGE]
+                            SHUFFLE [RS_118]
+                              PartitionCols:_col0
+                              Group By Operator [GBY_96] (rows=87121617 width=135)
+                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
+                              <-Reducer 19 [SIMPLE_EDGE]
+                                SHUFFLE [RS_95]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_94] (rows=174243235 width=135)
+                                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","sum(_col3)"],keys:_col6
+                                    Merge Join Operator [MERGEJOIN_185] (rows=174243235 width=135)
+                                      Conds:RS_90._col1=RS_91._col0(Inner),Output:["_col2","_col3","_col6"]
+                                    <-Map 30 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_91]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_86] (rows=4602 width=585)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_174] (rows=4602 width=585)
+                                            predicate:wp_web_page_sk is not null
+                                            TableScan [TS_84] (rows=4602 width=585)
+                                              default@web_page,web_page,Tbl:COMPLETE,Col:NONE,Output:["wp_web_page_sk"]
+                                    <-Reducer 18 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_90]
+                                        PartitionCols:_col1
+                                        Merge Join Operator [MERGEJOIN_184] (rows=158402938 width=135)
+                                          Conds:RS_87._col0=RS_88._col0(Inner),Output:["_col1","_col2","_col3"]
+                                        <-Map 9 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_88]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_83] (rows=8116 width=1119)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_173] (rows=8116 width=1119)
+                                                predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1998-08-04 00:00:00.0 AND 1998-09-03 00:00:00.0 and d_date_sk is not null)
+                                                 Please refer to the previous TableScan [TS_3]
+                                        <-Map 29 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_87]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_80] (rows=144002668 width=135)
+                                              Output:["_col0","_col1","_col2","_col3"]
+                                              Filter Operator [FIL_172] (rows=144002668 width=135)
+                                                predicate:(ws_sold_date_sk is not null and ws_web_page_sk is not null)
+                                                TableScan [TS_78] (rows=144002668 width=135)
+                                                  default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_web_page_sk","ws_ext_sales_price","ws_net_profit"]
+                          <-Reducer 24 [SIMPLE_EDGE]
+                            SHUFFLE [RS_119]
+                              PartitionCols:_col0
+                              Group By Operator [GBY_116] (rows=8711072 width=92)
+                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
+                              <-Reducer 23 [SIMPLE_EDGE]
+                                SHUFFLE [RS_115]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_114] (rows=17422145 width=92)
+                                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","sum(_col3)"],keys:_col6
+                                    Merge Join Operator [MERGEJOIN_187] (rows=17422145 width=92)
+                                      Conds:RS_110._col1=RS_111._col0(Inner),Output:["_col2","_col3","_col6"]
+                                    <-Map 30 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_111]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_106] (rows=4602 width=585)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_177] (rows=4602 width=585)
+                                            predicate:wp_web_page_sk is not null
+                                             Please refer to the previous TableScan [TS_84]
+                                    <-Reducer 22 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_110]
+                                        PartitionCols:_col1
+                                        Merge Join Operator [MERGEJOIN_186] (rows=15838314 width=92)
+                                          Conds:RS_107._col0=RS_108._col0(Inner),Output:["_col1","_col2","_col3"]
+                                        <-Map 9 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_108]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_103] (rows=8116 width=1119)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_176] (rows=8116 width=1119)
+                                                predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1998-08-04 00:00:00.0 AND 1998-09-03 00:00:00.0 and d_date_sk is not null)
+                                                 Please refer to the previous TableScan [TS_3]
+                                        <-Map 31 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_107]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_100] (rows=14398467 width=92)
+                                              Output:["_col0","_col1","_col2","_col3"]
+                                              Filter Operator [FIL_175] (rows=14398467 width=92)
+                                                predicate:(wr_returned_date_sk is not null and wr_web_page_sk is not null)
+                                                TableScan [TS_98] (rows=14398467 width=92)
+                                                  default@web_returns,web_returns,Tbl:COMPLETE,Col:NONE,Output:["wr_returned_date_sk","wr_web_page_sk","wr_return_amt","wr_net_loss"]
+                  <-Reducer 5 [CONTAINS]
+                    Reduce Output Operator [RS_125]
+                      PartitionCols:_col0, _col1, _col2
+                      Group By Operator [GBY_124] (rows=1912659936 width=163)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col2)","sum(_col3)","sum(_col4)"],keys:_col0, _col1, 0
+                        Select Operator [SEL_43] (rows=383325119 width=88)
+                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                          Merge Join Operator [MERGEJOIN_188] (rows=383325119 width=88)
+                            Conds:RS_40._col0=RS_41._col0(Left Outer),Output:["_col0","_col1","_col2","_col4","_col5"]
+                          <-Reducer 12 [SIMPLE_EDGE]
+                            SHUFFLE [RS_41]
+                              PartitionCols:_col0
+                              Group By Operator [GBY_38] (rows=34842647 width=77)
+                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
+                              <-Reducer 11 [SIMPLE_EDGE]
+                                SHUFFLE [RS_37]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_36] (rows=69685294 width=77)
+                                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","sum(_col3)"],keys:_col6
+                                    Merge Join Operator [MERGEJOIN_181] (rows=69685294 width=77)
+                                      Conds:RS_32._col1=RS_33._col0(Inner),Output:["_col2","_col3","_col6"]
+                                    <-Map 25 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_33]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_28] (rows=1704 width=1910)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_167] (rows=1704 width=1910)
+                                            predicate:s_store_sk is not null
+                                            TableScan [TS_6] (rows=1704 width=1910)
+                                              default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk"]
+                                    <-Reducer 10 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_32]
+                                        PartitionCols:_col1
+                                        Merge Join Operator [MERGEJOIN_180] (rows=63350266 width=77)
+                                          Conds:RS_29._col0=RS_30._col0(Inner),Output:["_col1","_col2","_col3"]
+                                        <-Map 9 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_30]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_25] (rows=8116 width=1119)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_166] (rows=8116 width=1119)
+                                                predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1998-08-04 00:00:00.0 AND 1998-09-03 00:00:00.0 and d_date_sk is not null)
+                                                 Please refer to the previous TableScan [TS_3]
+                                        <-Map 26 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_29]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_22] (rows=57591150 width=77)
+                                              Output:["_col0","_col1","_col2","_col3"]
+                                              Filter Operator [FIL_165] (rows=57591150 width=77)
+                                                predicate:(sr_returned_date_sk is not null and sr_store_sk is not null)
+                                                TableScan [TS_20] (rows=57591150 width=77)
+                                                  default@store_returns,store_returns,Tbl:COMPLETE,Col:NONE,Output:["sr_returned_date_sk","sr_store_sk","sr_return_amt","sr_net_loss"]
+                          <-Reducer 4 [SIMPLE_EDGE]
+                            SHUFFLE [RS_40]
+                              PartitionCols:_col0
+                              Group By Operator [GBY_18] (rows=348477374 width=88)
+                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
+                              <-Reducer 3 [SIMPLE_EDGE]
+                                SHUFFLE [RS_17]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_16] (rows=696954748 width=88)
+                                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","sum(_col3)"],keys:_col6
+                                    Merge Join Operator [MERGEJOIN_179] (rows=696954748 width=88)
+                                      Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col2","_col3","_col6"]
+                                    <-Map 25 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_13]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_8] (rows=1704 width=1910)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_164] (rows=1704 width=1910)
+                                            predicate:s_store_sk is not null
+                                             Please refer to the previous TableScan [TS_6]
+                                    <-Reducer 2 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_12]
+                                        PartitionCols:_col1
+                                        Merge Join Operator [MERGEJOIN_178] (rows=633595212 width=88)
+                                          Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col1","_col2","_col3"]
+                                        <-Map 9 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_10]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_5] (rows=8116 width=1119)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_163] (rows=8116 width=1119)
+                                                predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1998-08-04 00:00:00.0 AND 1998-09-03 00:00:00.0 and d_date_sk is not null)
+                                                 Please refer to the previous TableScan [TS_3]
+                                        <-Map 1 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_9]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_2] (rows=575995635 width=88)
+                                              Output:["_col0","_col1","_col2","_col3"]
+                                              Filter Operator [FIL_162] (rows=575995635 width=88)
+                                                predicate:(ss_sold_date_sk is not null and ss_store_sk is not null)
+                                                TableScan [TS_0] (rows=575995635 width=88)
+                                                  default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_store_sk","ss_ext_sales_price","ss_net_profit"]
+

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query78.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query78.q.out b/ql/src/test/results/clientpositive/perf/query78.q.out
new file mode 100644
index 0000000..8710085
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query78.q.out
@@ -0,0 +1,302 @@
+PREHOOK: query: explain
+with ws as
+  (select d_year AS ws_sold_year, ws_item_sk,
+    ws_bill_customer_sk ws_customer_sk,
+    sum(ws_quantity) ws_qty,
+    sum(ws_wholesale_cost) ws_wc,
+    sum(ws_sales_price) ws_sp
+   from web_sales
+   left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk
+   join date_dim on ws_sold_date_sk = d_date_sk
+   where wr_order_number is null
+   group by d_year, ws_item_sk, ws_bill_customer_sk
+   ),
+cs as
+  (select d_year AS cs_sold_year, cs_item_sk,
+    cs_bill_customer_sk cs_customer_sk,
+    sum(cs_quantity) cs_qty,
+    sum(cs_wholesale_cost) cs_wc,
+    sum(cs_sales_price) cs_sp
+   from catalog_sales
+   left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk
+   join date_dim on cs_sold_date_sk = d_date_sk
+   where cr_order_number is null
+   group by d_year, cs_item_sk, cs_bill_customer_sk
+   ),
+ss as
+  (select d_year AS ss_sold_year, ss_item_sk,
+    ss_customer_sk,
+    sum(ss_quantity) ss_qty,
+    sum(ss_wholesale_cost) ss_wc,
+    sum(ss_sales_price) ss_sp
+   from store_sales
+   left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+   join date_dim on ss_sold_date_sk = d_date_sk
+   where sr_ticket_number is null
+   group by d_year, ss_item_sk, ss_customer_sk
+   )
+ select 
+ss_sold_year, ss_item_sk, ss_customer_sk,
+round(ss_qty/(coalesce(ws_qty+cs_qty,1)),2) ratio,
+ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price,
+coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty,
+coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost,
+coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price
+from ss
+left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk)
+left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=cs_item_sk and cs_customer_sk=ss_customer_sk)
+where coalesce(ws_qty,0)>0 and coalesce(cs_qty, 0)>0 and ss_sold_year=2000
+order by 
+  ss_sold_year, ss_item_sk, ss_customer_sk,
+  ss_qty desc, ss_wc desc, ss_sp desc,
+  other_chan_qty,
+  other_chan_wholesale_cost,
+  other_chan_sales_price,
+  round(ss_qty/(coalesce(ws_qty+cs_qty,1)),2)
+limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+with ws as
+  (select d_year AS ws_sold_year, ws_item_sk,
+    ws_bill_customer_sk ws_customer_sk,
+    sum(ws_quantity) ws_qty,
+    sum(ws_wholesale_cost) ws_wc,
+    sum(ws_sales_price) ws_sp
+   from web_sales
+   left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk
+   join date_dim on ws_sold_date_sk = d_date_sk
+   where wr_order_number is null
+   group by d_year, ws_item_sk, ws_bill_customer_sk
+   ),
+cs as
+  (select d_year AS cs_sold_year, cs_item_sk,
+    cs_bill_customer_sk cs_customer_sk,
+    sum(cs_quantity) cs_qty,
+    sum(cs_wholesale_cost) cs_wc,
+    sum(cs_sales_price) cs_sp
+   from catalog_sales
+   left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk
+   join date_dim on cs_sold_date_sk = d_date_sk
+   where cr_order_number is null
+   group by d_year, cs_item_sk, cs_bill_customer_sk
+   ),
+ss as
+  (select d_year AS ss_sold_year, ss_item_sk,
+    ss_customer_sk,
+    sum(ss_quantity) ss_qty,
+    sum(ss_wholesale_cost) ss_wc,
+    sum(ss_sales_price) ss_sp
+   from store_sales
+   left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+   join date_dim on ss_sold_date_sk = d_date_sk
+   where sr_ticket_number is null
+   group by d_year, ss_item_sk, ss_customer_sk
+   )
+ select 
+ss_sold_year, ss_item_sk, ss_customer_sk,
+round(ss_qty/(coalesce(ws_qty+cs_qty,1)),2) ratio,
+ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price,
+coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty,
+coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost,
+coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price
+from ss
+left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk)
+left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=cs_item_sk and cs_customer_sk=ss_customer_sk)
+where coalesce(ws_qty,0)>0 and coalesce(cs_qty, 0)>0 and ss_sold_year=2000
+order by 
+  ss_sold_year, ss_item_sk, ss_customer_sk,
+  ss_qty desc, ss_wc desc, ss_sp desc,
+  other_chan_qty,
+  other_chan_wholesale_cost,
+  other_chan_sales_price,
+  round(ss_qty/(coalesce(ws_qty+cs_qty,1)),2)
+limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 10 <- Reducer 9 (SIMPLE_EDGE)
+Reducer 12 <- Map 11 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE)
+Reducer 15 <- Map 14 (SIMPLE_EDGE), Map 16 (SIMPLE_EDGE)
+Reducer 18 <- Map 17 (SIMPLE_EDGE), Map 19 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 10 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Map 1 (SIMPLE_EDGE), Reducer 15 (SIMPLE_EDGE)
+Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+Reducer 9 <- Map 1 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 6
+      File Output Operator [FS_79]
+        Select Operator [SEL_78] (rows=100 width=135)
+          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
+          Limit [LIM_77] (rows=100 width=135)
+            Number of rows:100
+            Select Operator [SEL_76] (rows=31942874 width=135)
+              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
+            <-Reducer 5 [SIMPLE_EDGE]
+              SHUFFLE [RS_75]
+                Select Operator [SEL_74] (rows=31942874 width=135)
+                  Output:["_col0","_col1","_col6","_col7","_col8","_col9","_col10","_col11","_col12"]
+                  Filter Operator [FIL_73] (rows=31942874 width=135)
+                    predicate:(COALESCE(_col11,0) > 0)
+                    Merge Join Operator [MERGEJOIN_114] (rows=95828623 width=135)
+                      Conds:RS_70._col1=RS_71._col0(Left Outer),Output:["_col0","_col1","_col2","_col3","_col4","_col7","_col8","_col9","_col11","_col12","_col13"]
+                    <-Reducer 10 [SIMPLE_EDGE]
+                      SHUFFLE [RS_71]
+                        PartitionCols:_col0
+                        Select Operator [SEL_68] (rows=87116929 width=135)
+                          Output:["_col0","_col1","_col2","_col3"]
+                          Group By Operator [GBY_67] (rows=87116929 width=135)
+                            Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"],keys:KEY._col0, KEY._col1
+                          <-Reducer 9 [SIMPLE_EDGE]
+                            SHUFFLE [RS_66]
+                              PartitionCols:_col0, _col1
+                              Group By Operator [GBY_65] (rows=174233858 width=135)
+                                Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col6)","sum(_col7)","sum(_col8)"],keys:_col3, _col4
+                                Merge Join Operator [MERGEJOIN_112] (rows=174233858 width=135)
+                                  Conds:RS_61._col0=RS_62._col0(Inner),Output:["_col3","_col4","_col6","_col7","_col8"]
+                                <-Map 1 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_61]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_49] (rows=36524 width=1119)
+                                      Output:["_col0"]
+                                      Filter Operator [FIL_104] (rows=36524 width=1119)
+                                        predicate:((d_year = 2000) and d_date_sk is not null)
+                                        TableScan [TS_0] (rows=73049 width=1119)
+                                          default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
+                                <-Reducer 18 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_62]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_60] (rows=158394413 width=135)
+                                      Output:["_col0","_col1","_col2","_col4","_col5","_col6"]
+                                      Filter Operator [FIL_59] (rows=158394413 width=135)
+                                        predicate:_col8 is null
+                                        Merge Join Operator [MERGEJOIN_111] (rows=316788826 width=135)
+                                          Conds:RS_56._col2, _col3=RS_57._col0, _col1(Left Outer),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8"]
+                                        <-Map 17 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_56]
+                                            PartitionCols:_col2, _col3
+                                            Select Operator [SEL_52] (rows=287989836 width=135)
+                                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                              Filter Operator [FIL_105] (rows=287989836 width=135)
+                                                predicate:(cs_sold_date_sk is not null and cs_item_sk is not null)
+                                                TableScan [TS_50] (rows=287989836 width=135)
+                                                  default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_bill_customer_sk","cs_item_sk","cs_order_number","cs_quantity","cs_wholesale_cost","cs_sales_price"]
+                                        <-Map 19 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_57]
+                                            PartitionCols:_col0, _col1
+                                            Select Operator [SEL_55] (rows=28798881 width=106)
+                                              Output:["_col0","_col1"]
+                                              Filter Operator [FIL_106] (rows=28798881 width=106)
+                                                predicate:cr_item_sk is not null
+                                                TableScan [TS_53] (rows=28798881 width=106)
+                                                  default@catalog_returns,catalog_returns,Tbl:COMPLETE,Col:NONE,Output:["cr_item_sk","cr_order_number"]
+                    <-Reducer 4 [SIMPLE_EDGE]
+                      SHUFFLE [RS_70]
+                        PartitionCols:_col1
+                        Filter Operator [FIL_45] (rows=63887519 width=88)
+                          predicate:(COALESCE(_col7,0) > 0)
+                          Merge Join Operator [MERGEJOIN_113] (rows=191662559 width=88)
+                            Conds:RS_42._col1, _col0=RS_43._col1, _col0(Left Outer),Output:["_col0","_col1","_col2","_col3","_col4","_col7","_col8","_col9"]
+                          <-Reducer 3 [SIMPLE_EDGE]
+                            SHUFFLE [RS_42]
+                              PartitionCols:_col1, _col0
+                              Select Operator [SEL_20] (rows=174238687 width=88)
+                                Output:["_col0","_col1","_col2","_col3","_col4"]
+                                Group By Operator [GBY_19] (rows=174238687 width=88)
+                                  Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"],keys:KEY._col0, KEY._col1
+                                <-Reducer 2 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_18]
+                                    PartitionCols:_col0, _col1
+                                    Group By Operator [GBY_17] (rows=348477374 width=88)
+                                      Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col6)","sum(_col7)","sum(_col8)"],keys:_col4, _col3
+                                      Merge Join Operator [MERGEJOIN_108] (rows=348477374 width=88)
+                                        Conds:RS_13._col0=RS_14._col0(Inner),Output:["_col3","_col4","_col6","_col7","_col8"]
+                                      <-Map 1 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_13]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_2] (rows=36524 width=1119)
+                                            Output:["_col0"]
+                                            Filter Operator [FIL_98] (rows=36524 width=1119)
+                                              predicate:((d_year = 2000) and d_date_sk is not null)
+                                               Please refer to the previous TableScan [TS_0]
+                                      <-Reducer 12 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_14]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_12] (rows=316797606 width=88)
+                                            Output:["_col0","_col1","_col2","_col4","_col5","_col6"]
+                                            Filter Operator [FIL_11] (rows=316797606 width=88)
+                                              predicate:_col8 is null
+                                              Merge Join Operator [MERGEJOIN_107] (rows=633595212 width=88)
+                                                Conds:RS_8._col1, _col3=RS_9._col0, _col1(Left Outer),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8"]
+                                              <-Map 11 [SIMPLE_EDGE]
+                                                SHUFFLE [RS_8]
+                                                  PartitionCols:_col1, _col3
+                                                  Select Operator [SEL_5] (rows=575995635 width=88)
+                                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                    Filter Operator [FIL_99] (rows=575995635 width=88)
+                                                      predicate:ss_sold_date_sk is not null
+                                                      TableScan [TS_3] (rows=575995635 width=88)
+                                                        default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_ticket_number","ss_quantity","ss_wholesale_cost","ss_sales_price"]
+                                              <-Map 13 [SIMPLE_EDGE]
+                                                SHUFFLE [RS_9]
+                                                  PartitionCols:_col0, _col1
+                                                  Select Operator [SEL_7] (rows=57591150 width=77)
+                                                    Output:["_col0","_col1"]
+                                                    TableScan [TS_6] (rows=57591150 width=77)
+                                                      default@store_returns,store_returns,Tbl:COMPLETE,Col:NONE,Output:["sr_item_sk","sr_ticket_number"]
+                          <-Reducer 8 [SIMPLE_EDGE]
+                            SHUFFLE [RS_43]
+                              PartitionCols:_col1, _col0
+                              Select Operator [SEL_41] (rows=43560808 width=135)
+                                Output:["_col0","_col1","_col2","_col3","_col4"]
+                                Group By Operator [GBY_40] (rows=43560808 width=135)
+                                  Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"],keys:KEY._col0, KEY._col1
+                                <-Reducer 7 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_39]
+                                    PartitionCols:_col0, _col1
+                                    Group By Operator [GBY_38] (rows=87121617 width=135)
+                                      Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col6)","sum(_col7)","sum(_col8)"],keys:_col4, _col3
+                                      Merge Join Operator [MERGEJOIN_110] (rows=87121617 width=135)
+                                        Conds:RS_34._col0=RS_35._col0(Inner),Output:["_col3","_col4","_col6","_col7","_col8"]
+                                      <-Map 1 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_34]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_23] (rows=36524 width=1119)
+                                            Output:["_col0"]
+                                            Filter Operator [FIL_101] (rows=36524 width=1119)
+                                              predicate:((d_year = 2000) and d_date_sk is not null)
+                                               Please refer to the previous TableScan [TS_0]
+                                      <-Reducer 15 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_35]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_33] (rows=79201469 width=135)
+                                            Output:["_col0","_col1","_col2","_col4","_col5","_col6"]
+                                            Filter Operator [FIL_32] (rows=79201469 width=135)
+                                              predicate:_col8 is null
+                                              Merge Join Operator [MERGEJOIN_109] (rows=158402938 width=135)
+                                                Conds:RS_29._col1, _col3=RS_30._col0, _col1(Left Outer),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8"]
+                                              <-Map 14 [SIMPLE_EDGE]
+                                                SHUFFLE [RS_29]
+                                                  PartitionCols:_col1, _col3
+                                                  Select Operator [SEL_26] (rows=144002668 width=135)
+                                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                    Filter Operator [FIL_102] (rows=144002668 width=135)
+                                                      predicate:ws_sold_date_sk is not null
+                                                      TableScan [TS_24] (rows=144002668 width=135)
+                                                        default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_bill_customer_sk","ws_order_number","ws_quantity","ws_wholesale_cost","ws_sales_price"]
+                                              <-Map 16 [SIMPLE_EDGE]
+                                                SHUFFLE [RS_30]
+                                                  PartitionCols:_col0, _col1
+                                                  Select Operator [SEL_28] (rows=14398467 width=92)
+                                                    Output:["_col0","_col1"]
+                                                    TableScan [TS_27] (rows=14398467 width=92)
+                                                      default@web_returns,web_returns,Tbl:COMPLETE,Col:NONE,Output:["wr_item_sk","wr_order_number"]
+

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query79.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query79.q.out b/ql/src/test/results/clientpositive/perf/query79.q.out
index 4955745..18b32e0 100644
--- a/ql/src/test/results/clientpositive/perf/query79.q.out
+++ b/ql/src/test/results/clientpositive/perf/query79.q.out
@@ -1,6 +1,46 @@
-PREHOOK: query: explain select c_last_name,c_first_name,substr(s_city,1,30) sub,ss_ticket_number,amt,profit from (select ss_ticket_number ,ss_customer_sk ,store.s_city ,sum(ss_coupon_amt) amt ,sum(ss_net_profit) profit from store_sales,date_dim,store,household_demographics where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and (household_demographics.hd_dep_count = 8 or household_demographics.hd_vehicle_count > 0) and date_dim.d_dow = 1 and date_dim.d_year in (1998,1998+1,1998+2) and store.s_number_employees between 200 and 295 group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer where ms.ss_customer_sk = customer.c_customer_sk order by c_last_name,c_first_name,sub, profit limit 100
+PREHOOK: query: explain
+select 
+  c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit
+  from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,store.s_city
+          ,sum(ss_coupon_amt) amt
+          ,sum(ss_net_profit) profit
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and (household_demographics.hd_dep_count = 8 or household_demographics.hd_vehicle_count > 0)
+    and date_dim.d_dow = 1
+    and date_dim.d_year in (1998,1998+1,1998+2) 
+    and store.s_number_employees between 200 and 295
+    group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer
+    where ss_customer_sk = c_customer_sk
+ order by c_last_name,c_first_name,substr(s_city,1,30), profit
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select c_last_name,c_first_name,substr(s_city,1,30) sub,ss_ticket_number,amt,profit from (select ss_ticket_number ,ss_customer_sk ,store.s_city ,sum(ss_coupon_amt) amt ,sum(ss_net_profit) profit from store_sales,date_dim,store,household_demographics where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and (household_demographics.hd_dep_count = 8 or household_demographics.hd_vehicle_count > 0) and date_dim.d_dow = 1 and date_dim.d_year in (1998,1998+1,1998+2) and store.s_number_employees between 200 and 295 group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer where ms.ss_customer_sk = customer.c_customer_sk order by c_last_name,c_first_name,sub, profit limit 100
+POSTHOOK: query: explain
+select 
+  c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit
+  from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,store.s_city
+          ,sum(ss_coupon_amt) amt
+          ,sum(ss_net_profit) profit
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and (household_demographics.hd_dep_count = 8 or household_demographics.hd_vehicle_count > 0)
+    and date_dim.d_dow = 1
+    and date_dim.d_year in (1998,1998+1,1998+2) 
+    and store.s_number_employees between 200 and 295
+    group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer
+    where ss_customer_sk = c_customer_sk
+ order by c_last_name,c_first_name,substr(s_city,1,30), profit
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -14,10 +54,10 @@ Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
       Reducer 3
-      File Output Operator [FS_37]
+      File Output Operator [FS_38]
         Limit [LIM_36] (rows=100 width=88)
           Number of rows:100
           Select Operator [SEL_35] (rows=421657640 width=88)
@@ -25,15 +65,15 @@ Stage-0
           <-Reducer 2 [SIMPLE_EDGE]
             SHUFFLE [RS_34]
               Select Operator [SEL_33] (rows=421657640 width=88)
-                Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                Merge Join Operator [MERGEJOIN_60] (rows=421657640 width=88)
+                Output:["_col0","_col1","_col3","_col4","_col5","_col6"]
+                Merge Join Operator [MERGEJOIN_61] (rows=421657640 width=88)
                   Conds:RS_30._col0=RS_31._col1(Inner),Output:["_col1","_col2","_col3","_col5","_col6","_col7"]
                 <-Map 1 [SIMPLE_EDGE]
                   SHUFFLE [RS_30]
                     PartitionCols:_col0
                     Select Operator [SEL_2] (rows=80000000 width=860)
                       Output:["_col0","_col1","_col2"]
-                      Filter Operator [FIL_52] (rows=80000000 width=860)
+                      Filter Operator [FIL_53] (rows=80000000 width=860)
                         predicate:c_customer_sk is not null
                         TableScan [TS_0] (rows=80000000 width=860)
                           default@customer,customer,Tbl:COMPLETE,Col:NONE,Output:["c_customer_sk","c_first_name","c_last_name"]
@@ -49,42 +89,42 @@ Stage-0
                           PartitionCols:_col0, _col1, _col2, _col3
                           Group By Operator [GBY_25] (rows=766650239 width=88)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col6)","sum(_col7)"],keys:_col1, _col3, _col5, _col13
-                            Merge Join Operator [MERGEJOIN_59] (rows=766650239 width=88)
+                            Merge Join Operator [MERGEJOIN_60] (rows=766650239 width=88)
                               Conds:RS_21._col2=RS_22._col0(Inner),Output:["_col1","_col3","_col5","_col6","_col7","_col13"]
                             <-Map 11 [SIMPLE_EDGE]
                               SHUFFLE [RS_22]
                                 PartitionCols:_col0
                                 Select Operator [SEL_14] (rows=6000 width=107)
                                   Output:["_col0"]
-                                  Filter Operator [FIL_56] (rows=6000 width=107)
+                                  Filter Operator [FIL_57] (rows=6000 width=107)
                                     predicate:(((hd_dep_count = 8) or (hd_vehicle_count > 0)) and hd_demo_sk is not null)
                                     TableScan [TS_12] (rows=7200 width=107)
                                       default@household_demographics,household_demographics,Tbl:COMPLETE,Col:NONE,Output:["hd_demo_sk","hd_dep_count","hd_vehicle_count"]
                             <-Reducer 6 [SIMPLE_EDGE]
                               SHUFFLE [RS_21]
                                 PartitionCols:_col2
-                                Merge Join Operator [MERGEJOIN_58] (rows=696954748 width=88)
+                                Merge Join Operator [MERGEJOIN_59] (rows=696954748 width=88)
                                   Conds:RS_18._col4=RS_19._col0(Inner),Output:["_col1","_col2","_col3","_col5","_col6","_col7","_col13"]
                                 <-Map 10 [SIMPLE_EDGE]
                                   SHUFFLE [RS_19]
                                     PartitionCols:_col0
                                     Select Operator [SEL_11] (rows=189 width=1910)
                                       Output:["_col0","_col2"]
-                                      Filter Operator [FIL_55] (rows=189 width=1910)
+                                      Filter Operator [FIL_56] (rows=189 width=1910)
                                         predicate:(s_number_employees BETWEEN 200 AND 295 and s_store_sk is not null)
                                         TableScan [TS_9] (rows=1704 width=1910)
                                           default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_number_employees","s_city"]
                                 <-Reducer 5 [SIMPLE_EDGE]
                                   SHUFFLE [RS_18]
                                     PartitionCols:_col4
-                                    Merge Join Operator [MERGEJOIN_57] (rows=633595212 width=88)
+                                    Merge Join Operator [MERGEJOIN_58] (rows=633595212 width=88)
                                       Conds:RS_15._col0=RS_16._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
                                     <-Map 4 [SIMPLE_EDGE]
                                       SHUFFLE [RS_15]
                                         PartitionCols:_col0
                                         Select Operator [SEL_5] (rows=575995635 width=88)
                                           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
-                                          Filter Operator [FIL_53] (rows=575995635 width=88)
+                                          Filter Operator [FIL_54] (rows=575995635 width=88)
                                             predicate:(ss_sold_date_sk is not null and ss_store_sk is not null and ss_hdemo_sk is not null and ss_customer_sk is not null)
                                             TableScan [TS_3] (rows=575995635 width=88)
                                               default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_customer_sk","ss_hdemo_sk","ss_addr_sk","ss_store_sk","ss_ticket_number","ss_coupon_amt","ss_net_profit"]
@@ -93,7 +133,7 @@ Stage-0
                                         PartitionCols:_col0
                                         Select Operator [SEL_8] (rows=18262 width=1119)
                                           Output:["_col0"]
-                                          Filter Operator [FIL_54] (rows=18262 width=1119)
+                                          Filter Operator [FIL_55] (rows=18262 width=1119)
                                             predicate:((d_year) IN (1998, 1999, 2000) and (d_dow = 1) 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_dow"]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query81.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query81.q.out b/ql/src/test/results/clientpositive/perf/query81.q.out
index 3acee4d..abeb577 100644
--- a/ql/src/test/results/clientpositive/perf/query81.q.out
+++ b/ql/src/test/results/clientpositive/perf/query81.q.out
@@ -1,4 +1,5 @@
-PREHOOK: query: explain with customer_total_return as
+PREHOOK: query: explain
+with customer_total_return as
  (select cr_returning_customer_sk as ctr_customer_sk
         ,ca_state as ctr_state, 
  	sum(cr_return_amt_inc_tax) as ctr_total_return
@@ -27,7 +28,8 @@ PREHOOK: query: explain with customer_total_return as
                   ,ca_location_type,ctr_total_return
  limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with customer_total_return as
+POSTHOOK: query: explain
+with customer_total_return as
  (select cr_returning_customer_sk as ctr_customer_sk
         ,ca_state as ctr_state, 
  	sum(cr_return_amt_inc_tax) as ctr_total_return

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query83.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query83.q.out b/ql/src/test/results/clientpositive/perf/query83.q.out
index f96306b..396c423 100644
--- a/ql/src/test/results/clientpositive/perf/query83.q.out
+++ b/ql/src/test/results/clientpositive/perf/query83.q.out
@@ -1,4 +1,5 @@
-PREHOOK: query: explain with sr_items as
+PREHOOK: query: explain
+with sr_items as
  (select i_item_id item_id,
         sum(sr_return_quantity) sr_item_qty
  from store_returns,
@@ -63,7 +64,8 @@ PREHOOK: query: explain with sr_items as
          ,sr_item_qty
  limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with sr_items as
+POSTHOOK: query: explain
+with sr_items as
  (select i_item_id item_id,
         sum(sr_return_quantity) sr_item_qty
  from store_returns,

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query84.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query84.q.out b/ql/src/test/results/clientpositive/perf/query84.q.out
index cf68729..f23cddd 100644
--- a/ql/src/test/results/clientpositive/perf/query84.q.out
+++ b/ql/src/test/results/clientpositive/perf/query84.q.out
@@ -1,6 +1,42 @@
-PREHOOK: query: explain select c_customer_id as customer_id ,concat(c_last_name, ', ', c_first_name) as customername from customer ,customer_address ,customer_demographics ,household_demographics ,income_band ,store_returns where ca_city = 'Hopewell' and customer.c_current_addr_sk = customer_address.ca_address_sk and ib_lower_bound >= 32287 and ib_upper_bound <= 32287 + 50000 and income_band.ib_income_band_sk = household_demographics.hd_income_band_sk and customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk and household_demographics.hd_demo_sk = customer.c_current_hdemo_sk and store_returns.sr_cdemo_sk = customer_demographics.cd_demo_sk order by customer_id limit 100
+PREHOOK: query: explain
+select  c_customer_id as customer_id
+       ,c_last_name || ', ' || c_first_name as customername
+ from customer
+     ,customer_address
+     ,customer_demographics
+     ,household_demographics
+     ,income_band
+     ,store_returns
+ where ca_city	        =  'Hopewell'
+   and c_current_addr_sk = ca_address_sk
+   and ib_lower_bound   >=  32287
+   and ib_upper_bound   <=  32287 + 50000
+   and ib_income_band_sk = hd_income_band_sk
+   and cd_demo_sk = c_current_cdemo_sk
+   and hd_demo_sk = c_current_hdemo_sk
+   and sr_cdemo_sk = cd_demo_sk
+ order by c_customer_id
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select c_customer_id as customer_id ,concat(c_last_name, ', ', c_first_name) as customername from customer ,customer_address ,customer_demographics ,household_demographics ,income_band ,store_returns where ca_city = 'Hopewell' and customer.c_current_addr_sk = customer_address.ca_address_sk and ib_lower_bound >= 32287 and ib_upper_bound <= 32287 + 50000 and income_band.ib_income_band_sk = household_demographics.hd_income_band_sk and customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk and household_demographics.hd_demo_sk = customer.c_current_hdemo_sk and store_returns.sr_cdemo_sk = customer_demographics.cd_demo_sk order by customer_id limit 100
+POSTHOOK: query: explain
+select  c_customer_id as customer_id
+       ,c_last_name || ', ' || c_first_name as customername
+ from customer
+     ,customer_address
+     ,customer_demographics
+     ,household_demographics
+     ,income_band
+     ,store_returns
+ where ca_city	        =  'Hopewell'
+   and c_current_addr_sk = ca_address_sk
+   and ib_lower_bound   >=  32287
+   and ib_upper_bound   <=  32287 + 50000
+   and ib_income_band_sk = hd_income_band_sk
+   and cd_demo_sk = c_current_cdemo_sk
+   and hd_demo_sk = c_current_hdemo_sk
+   and sr_cdemo_sk = cd_demo_sk
+ order by c_customer_id
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -13,10 +49,10 @@ Reducer 7 <- Reducer 10 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
       Reducer 3
-      File Output Operator [FS_37]
+      File Output Operator [FS_38]
         Limit [LIM_36] (rows=100 width=860)
           Number of rows:100
           Select Operator [SEL_35] (rows=212960011 width=860)
@@ -24,15 +60,15 @@ Stage-0
           <-Reducer 2 [SIMPLE_EDGE]
             SHUFFLE [RS_34]
               Select Operator [SEL_33] (rows=212960011 width=860)
-                Output:["_col0","_col1"]
-                Merge Join Operator [MERGEJOIN_67] (rows=212960011 width=860)
+                Output:["_col1","_col2"]
+                Merge Join Operator [MERGEJOIN_68] (rows=212960011 width=860)
                   Conds:RS_29._col0=RS_30._col0(Inner),RS_30._col0=RS_31._col1(Inner),Output:["_col2","_col6","_col7"]
                 <-Map 1 [SIMPLE_EDGE]
                   SHUFFLE [RS_29]
                     PartitionCols:_col0
                     Select Operator [SEL_2] (rows=57591150 width=77)
                       Output:["_col0"]
-                      Filter Operator [FIL_58] (rows=57591150 width=77)
+                      Filter Operator [FIL_59] (rows=57591150 width=77)
                         predicate:sr_cdemo_sk is not null
                         TableScan [TS_0] (rows=57591150 width=77)
                           default@store_returns,store_returns,Tbl:COMPLETE,Col:NONE,Output:["sr_cdemo_sk"]
@@ -41,26 +77,26 @@ Stage-0
                     PartitionCols:_col0
                     Select Operator [SEL_5] (rows=1861800 width=385)
                       Output:["_col0"]
-                      Filter Operator [FIL_59] (rows=1861800 width=385)
+                      Filter Operator [FIL_60] (rows=1861800 width=385)
                         predicate:cd_demo_sk is not null
                         TableScan [TS_3] (rows=1861800 width=385)
                           default@customer_demographics,customer_demographics,Tbl:COMPLETE,Col:NONE,Output:["cd_demo_sk"]
                 <-Reducer 7 [SIMPLE_EDGE]
                   SHUFFLE [RS_31]
                     PartitionCols:_col1
-                    Merge Join Operator [MERGEJOIN_66] (rows=96800003 width=860)
+                    Merge Join Operator [MERGEJOIN_67] (rows=96800003 width=860)
                       Conds:RS_25._col2=RS_26._col0(Inner),Output:["_col0","_col1","_col4","_col5"]
                     <-Reducer 10 [SIMPLE_EDGE]
                       SHUFFLE [RS_26]
                         PartitionCols:_col0
-                        Merge Join Operator [MERGEJOIN_65] (rows=7920 width=107)
+                        Merge Join Operator [MERGEJOIN_66] (rows=7920 width=107)
                           Conds:RS_18._col1=RS_19._col0(Inner),Output:["_col0"]
                         <-Map 11 [SIMPLE_EDGE]
                           SHUFFLE [RS_19]
                             PartitionCols:_col0
                             Select Operator [SEL_17] (rows=2 width=12)
                               Output:["_col0"]
-                              Filter Operator [FIL_63] (rows=2 width=12)
+                              Filter Operator [FIL_64] (rows=2 width=12)
                                 predicate:((ib_lower_bound >= 32287) and (ib_upper_bound <= 82287) and ib_income_band_sk is not null)
                                 TableScan [TS_15] (rows=20 width=12)
                                   default@income_band,income_band,Tbl:COMPLETE,Col:NONE,Output:["ib_income_band_sk","ib_lower_bound","ib_upper_bound"]
@@ -69,21 +105,21 @@ Stage-0
                             PartitionCols:_col1
                             Select Operator [SEL_14] (rows=7200 width=107)
                               Output:["_col0","_col1"]
-                              Filter Operator [FIL_62] (rows=7200 width=107)
+                              Filter Operator [FIL_63] (rows=7200 width=107)
                                 predicate:(hd_demo_sk is not null and hd_income_band_sk is not null)
                                 TableScan [TS_12] (rows=7200 width=107)
                                   default@household_demographics,household_demographics,Tbl:COMPLETE,Col:NONE,Output:["hd_demo_sk","hd_income_band_sk"]
                     <-Reducer 6 [SIMPLE_EDGE]
                       SHUFFLE [RS_25]
                         PartitionCols:_col2
-                        Merge Join Operator [MERGEJOIN_64] (rows=88000001 width=860)
+                        Merge Join Operator [MERGEJOIN_65] (rows=88000001 width=860)
                           Conds:RS_22._col3=RS_23._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col5"]
                         <-Map 5 [SIMPLE_EDGE]
                           SHUFFLE [RS_22]
                             PartitionCols:_col3
                             Select Operator [SEL_8] (rows=80000000 width=860)
                               Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                              Filter Operator [FIL_60] (rows=80000000 width=860)
+                              Filter Operator [FIL_61] (rows=80000000 width=860)
                                 predicate:(c_current_addr_sk is not null and c_current_cdemo_sk is not null and c_current_hdemo_sk is not null)
                                 TableScan [TS_6] (rows=80000000 width=860)
                                   default@customer,customer,Tbl:COMPLETE,Col:NONE,Output:["c_customer_id","c_current_cdemo_sk","c_current_hdemo_sk","c_current_addr_sk","c_first_name","c_last_name"]
@@ -92,7 +128,7 @@ Stage-0
                             PartitionCols:_col0
                             Select Operator [SEL_11] (rows=20000000 width=1014)
                               Output:["_col0"]
-                              Filter Operator [FIL_61] (rows=20000000 width=1014)
+                              Filter Operator [FIL_62] (rows=20000000 width=1014)
                                 predicate:((ca_city = 'Hopewell') and ca_address_sk is not null)
                                 TableScan [TS_9] (rows=40000000 width=1014)
                                   default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_city"]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query85.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query85.q.out b/ql/src/test/results/clientpositive/perf/query85.q.out
index 82df27b..86b961b 100644
--- a/ql/src/test/results/clientpositive/perf/query85.q.out
+++ b/ql/src/test/results/clientpositive/perf/query85.q.out
@@ -1,6 +1,168 @@
-PREHOOK: query: explain select substr(r_reason_desc,1,20) as r ,avg(ws_quantity) wq ,avg(wr_refunded_cash) ref ,avg(wr_fee) fee from web_sales, web_returns, web_page, customer_demographics cd1, customer_demographics cd2, customer_address, date_dim, reason where web_sales.ws_web_page_sk = web_page.wp_web_page_sk and web_sales.ws_item_sk = web_returns.wr_item_sk and web_sales.ws_order_number = web_returns.wr_order_number and web_sales.ws_sold_date_sk = date_dim.d_date_sk and d_year = 1998 and cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk and cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk and customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk and reason.r_reason_sk = web_returns.wr_reason_sk and ( ( cd1.cd_marital_status = 'M' and cd1.cd_marital_status = cd2.cd_marital_status and cd1.cd_education_status = '4 yr Degree' and cd1.cd_education_status = cd2.cd_education_status and ws_sales_price between 100.00 and 150.00 ) or ( cd1.cd_marital_status = 'D' and cd1.cd_ma
 rital_status = cd2.cd_marital_status and cd1.cd_education_status = 'Primary' and cd1.cd_education_status = cd2.cd_education_status and ws_sales_price between 50.00 and 100.00 ) or ( cd1.cd_marital_status = 'U' and cd1.cd_marital_status = cd2.cd_marital_status and cd1.cd_education_status = 'Advanced Degree' and cd1.cd_education_status = cd2.cd_education_status and ws_sales_price between 150.00 and 200.00 ) ) and ( ( ca_country = 'United States' and ca_state in ('KY', 'GA', 'NM') and ws_net_profit between 100 and 200 ) or ( ca_country = 'United States' and ca_state in ('MT', 'OR', 'IN') and ws_net_profit between 150 and 300 ) or ( ca_country = 'United States' and ca_state in ('WI', 'MO', 'WV') and ws_net_profit between 50 and 250 ) ) group by r_reason_desc order by r, wq, ref, fee limit 100
+PREHOOK: query: explain
+select  substr(r_reason_desc,1,20)
+       ,avg(ws_quantity)
+       ,avg(wr_refunded_cash)
+       ,avg(wr_fee)
+ from web_sales, web_returns, web_page, customer_demographics cd1,
+      customer_demographics cd2, customer_address, date_dim, reason 
+ where ws_web_page_sk = wp_web_page_sk
+   and ws_item_sk = wr_item_sk
+   and ws_order_number = wr_order_number
+   and ws_sold_date_sk = d_date_sk and d_year = 1998
+   and cd1.cd_demo_sk = wr_refunded_cdemo_sk 
+   and cd2.cd_demo_sk = wr_returning_cdemo_sk
+   and ca_address_sk = wr_refunded_addr_sk
+   and r_reason_sk = wr_reason_sk
+   and
+   (
+    (
+     cd1.cd_marital_status = 'M'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = '4 yr Degree'
+     and 
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 100.00 and 150.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'D'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Primary' 
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 50.00 and 100.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'U'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Advanced Degree'
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 150.00 and 200.00
+    )
+   )
+   and
+   (
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('KY', 'GA', 'NM')
+     and ws_net_profit between 100 and 200  
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('MT', 'OR', 'IN')
+     and ws_net_profit between 150 and 300  
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('WI', 'MO', 'WV')
+     and ws_net_profit between 50 and 250  
+    )
+   )
+group by r_reason_desc
+order by substr(r_reason_desc,1,20)
+        ,avg(ws_quantity)
+        ,avg(wr_refunded_cash)
+        ,avg(wr_fee)
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select substr(r_reason_desc,1,20) as r ,avg(ws_quantity) wq ,avg(wr_refunded_cash) ref ,avg(wr_fee) fee from web_sales, web_returns, web_page, customer_demographics cd1, customer_demographics cd2, customer_address, date_dim, reason where web_sales.ws_web_page_sk = web_page.wp_web_page_sk and web_sales.ws_item_sk = web_returns.wr_item_sk and web_sales.ws_order_number = web_returns.wr_order_number and web_sales.ws_sold_date_sk = date_dim.d_date_sk and d_year = 1998 and cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk and cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk and customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk and reason.r_reason_sk = web_returns.wr_reason_sk and ( ( cd1.cd_marital_status = 'M' and cd1.cd_marital_status = cd2.cd_marital_status and cd1.cd_education_status = '4 yr Degree' and cd1.cd_education_status = cd2.cd_education_status and ws_sales_price between 100.00 and 150.00 ) or ( cd1.cd_marital_status = 'D' and cd1.cd_m
 arital_status = cd2.cd_marital_status and cd1.cd_education_status = 'Primary' and cd1.cd_education_status = cd2.cd_education_status and ws_sales_price between 50.00 and 100.00 ) or ( cd1.cd_marital_status = 'U' and cd1.cd_marital_status = cd2.cd_marital_status and cd1.cd_education_status = 'Advanced Degree' and cd1.cd_education_status = cd2.cd_education_status and ws_sales_price between 150.00 and 200.00 ) ) and ( ( ca_country = 'United States' and ca_state in ('KY', 'GA', 'NM') and ws_net_profit between 100 and 200 ) or ( ca_country = 'United States' and ca_state in ('MT', 'OR', 'IN') and ws_net_profit between 150 and 300 ) or ( ca_country = 'United States' and ca_state in ('WI', 'MO', 'WV') and ws_net_profit between 50 and 250 ) ) group by r_reason_desc order by r, wq, ref, fee limit 100
+POSTHOOK: query: explain
+select  substr(r_reason_desc,1,20)
+       ,avg(ws_quantity)
+       ,avg(wr_refunded_cash)
+       ,avg(wr_fee)
+ from web_sales, web_returns, web_page, customer_demographics cd1,
+      customer_demographics cd2, customer_address, date_dim, reason 
+ where ws_web_page_sk = wp_web_page_sk
+   and ws_item_sk = wr_item_sk
+   and ws_order_number = wr_order_number
+   and ws_sold_date_sk = d_date_sk and d_year = 1998
+   and cd1.cd_demo_sk = wr_refunded_cdemo_sk 
+   and cd2.cd_demo_sk = wr_returning_cdemo_sk
+   and ca_address_sk = wr_refunded_addr_sk
+   and r_reason_sk = wr_reason_sk
+   and
+   (
+    (
+     cd1.cd_marital_status = 'M'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = '4 yr Degree'
+     and 
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 100.00 and 150.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'D'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Primary' 
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 50.00 and 100.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'U'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Advanced Degree'
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 150.00 and 200.00
+    )
+   )
+   and
+   (
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('KY', 'GA', 'NM')
+     and ws_net_profit between 100 and 200  
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('MT', 'OR', 'IN')
+     and ws_net_profit between 150 and 300  
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('WI', 'MO', 'WV')
+     and ws_net_profit between 50 and 250  
+    )
+   )
+group by r_reason_desc
+order by substr(r_reason_desc,1,20)
+        ,avg(ws_quantity)
+        ,avg(wr_refunded_cash)
+        ,avg(wr_fee)
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -28,7 +190,7 @@ Stage-0
           <-Reducer 9 [SIMPLE_EDGE]
             SHUFFLE [RS_52]
               Select Operator [SEL_51] (rows=1023990 width=385)
-                Output:["_col0","_col1","_col2","_col3"]
+                Output:["_col5","_col6","_col7","_col8"]
                 Group By Operator [GBY_50] (rows=1023990 width=385)
                   Output:["_col0","_col1","_col2","_col3"],aggregations:["avg(VALUE._col0)","avg(VALUE._col1)","avg(VALUE._col2)"],keys:KEY._col0
                 <-Reducer 8 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query86.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query86.q.out b/ql/src/test/results/clientpositive/perf/query86.q.out
index bc86291..16f2cb9 100644
--- a/ql/src/test/results/clientpositive/perf/query86.q.out
+++ b/ql/src/test/results/clientpositive/perf/query86.q.out
@@ -5,9 +5,9 @@ select
    ,i_class
    ,grouping(i_category)+grouping(i_class) as lochierarchy
    ,rank() over (
-     partition by grouping(i_category)+grouping(i_class),
-     case when grouping(i_class) = 0 then i_category end 
-     order by sum(ws_net_paid) desc) as rank_within_parent
+ 	partition by grouping(i_category)+grouping(i_class),
+ 	case when grouping(i_class) = 0 then i_category end 
+ 	order by sum(ws_net_paid) desc) as rank_within_parent
  from
     web_sales
    ,date_dim       d1
@@ -30,9 +30,9 @@ select
    ,i_class
    ,grouping(i_category)+grouping(i_class) as lochierarchy
    ,rank() over (
-     partition by grouping(i_category)+grouping(i_class),
-     case when grouping(i_class) = 0 then i_category end 
-     order by sum(ws_net_paid) desc) as rank_within_parent
+ 	partition by grouping(i_category)+grouping(i_class),
+ 	case when grouping(i_class) = 0 then i_category end 
+ 	order by sum(ws_net_paid) desc) as rank_within_parent
  from
     web_sales
    ,date_dim       d1


[11/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query34.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query34.q.out b/ql/src/test/results/clientpositive/perf/query34.q.out
index 51710bd..a535ce5 100644
--- a/ql/src/test/results/clientpositive/perf/query34.q.out
+++ b/ql/src/test/results/clientpositive/perf/query34.q.out
@@ -1,6 +1,62 @@
-PREHOOK: query: explain select c_last_name ,c_first_name ,c_salutation ,c_preferred_cust_flag ,ss_ticket_number ,cnt from (select ss_ticket_number ,ss_customer_sk ,count(*) cnt from store_sales,date_dim,store,household_demographics where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) and (household_demographics.hd_buy_potential = '1001-5000' or household_demographics.hd_buy_potential = '5001-10000') and household_demographics.hd_vehicle_count > 0 and (case when household_demographics.hd_vehicle_count > 0 then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end) > 1.2 and date_dim.d_year in (1998,1998+1,1998+2) and store.s_county in ('Kittitas County','Adams County','Richland County','Furnas County', 'Orange County','Appanoose County','Franklin Parish','Tehama Co
 unty') group by ss_ticket_number,ss_customer_sk) dn,customer where dn.ss_customer_sk = customer.c_customer_sk and cnt between 15 and 20 order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc
+PREHOOK: query: explain
+select c_last_name
+       ,c_first_name
+       ,c_salutation
+       ,c_preferred_cust_flag
+       ,ss_ticket_number
+       ,cnt from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,count(*) cnt
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28)
+    and (household_demographics.hd_buy_potential = '>10000' or
+         household_demographics.hd_buy_potential = 'unknown')
+    and household_demographics.hd_vehicle_count > 0
+    and (case when household_demographics.hd_vehicle_count > 0 
+	then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count 
+	else null 
+	end)  > 1.2
+    and date_dim.d_year in (2000,2000+1,2000+2)
+    and store.s_county in ('Mobile County','Maverick County','Huron County','Kittitas County',
+                           'Fairfield County','Jackson County','Barrow County','Pennington County')
+    group by ss_ticket_number,ss_customer_sk) dn,customer
+    where ss_customer_sk = c_customer_sk
+      and cnt between 15 and 20
+    order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select c_last_name ,c_first_name ,c_salutation ,c_preferred_cust_flag ,ss_ticket_number ,cnt from (select ss_ticket_number ,ss_customer_sk ,count(*) cnt from store_sales,date_dim,store,household_demographics where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) and (household_demographics.hd_buy_potential = '1001-5000' or household_demographics.hd_buy_potential = '5001-10000') and household_demographics.hd_vehicle_count > 0 and (case when household_demographics.hd_vehicle_count > 0 then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end) > 1.2 and date_dim.d_year in (1998,1998+1,1998+2) and store.s_county in ('Kittitas County','Adams County','Richland County','Furnas County', 'Orange County','Appanoose County','Franklin Parish','Tehama C
 ounty') group by ss_ticket_number,ss_customer_sk) dn,customer where dn.ss_customer_sk = customer.c_customer_sk and cnt between 15 and 20 order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc
+POSTHOOK: query: explain
+select c_last_name
+       ,c_first_name
+       ,c_salutation
+       ,c_preferred_cust_flag
+       ,ss_ticket_number
+       ,cnt from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,count(*) cnt
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28)
+    and (household_demographics.hd_buy_potential = '>10000' or
+         household_demographics.hd_buy_potential = 'unknown')
+    and household_demographics.hd_vehicle_count > 0
+    and (case when household_demographics.hd_vehicle_count > 0 
+	then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count 
+	else null 
+	end)  > 1.2
+    and date_dim.d_year in (2000,2000+1,2000+2)
+    and store.s_county in ('Mobile County','Maverick County','Huron County','Kittitas County',
+                           'Fairfield County','Jackson County','Barrow County','Pennington County')
+    group by ss_ticket_number,ss_customer_sk) dn,customer
+    where ss_customer_sk = c_customer_sk
+      and cnt between 15 and 20
+    order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -57,7 +113,7 @@ Stage-0
                                 Select Operator [SEL_14] (rows=852 width=1910)
                                   Output:["_col0"]
                                   Filter Operator [FIL_56] (rows=852 width=1910)
-                                    predicate:((s_county) IN ('Kittitas County', 'Adams County', 'Richland County', 'Furnas County', 'Orange County', 'Appanoose County', 'Franklin Parish', 'Tehama County') and s_store_sk is not null)
+                                    predicate:((s_county) IN ('Mobile County', 'Maverick County', 'Huron County', 'Kittitas County', 'Fairfield County', 'Jackson County', 'Barrow County', 'Pennington County') and s_store_sk is not null)
                                     TableScan [TS_12] (rows=1704 width=1910)
                                       default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_county"]
                             <-Reducer 6 [SIMPLE_EDGE]
@@ -71,7 +127,7 @@ Stage-0
                                     Select Operator [SEL_11] (rows=1200 width=107)
                                       Output:["_col0"]
                                       Filter Operator [FIL_55] (rows=1200 width=107)
-                                        predicate:(((hd_buy_potential = '1001-5000') or (hd_buy_potential = '5001-10000')) and (hd_vehicle_count > 0) and CASE WHEN ((hd_vehicle_count > 0)) THEN (((UDFToDouble(hd_dep_count) / UDFToDouble(hd_vehicle_count)) > 1.2)) ELSE (null) END and hd_demo_sk is not null)
+                                        predicate:(((hd_buy_potential = '>10000') or (hd_buy_potential = 'unknown')) and (hd_vehicle_count > 0) and CASE WHEN ((hd_vehicle_count > 0)) THEN (((UDFToDouble(hd_dep_count) / UDFToDouble(hd_vehicle_count)) > 1.2)) ELSE (null) END and hd_demo_sk is not null)
                                         TableScan [TS_9] (rows=7200 width=107)
                                           default@household_demographics,household_demographics,Tbl:COMPLETE,Col:NONE,Output:["hd_demo_sk","hd_buy_potential","hd_dep_count","hd_vehicle_count"]
                                 <-Reducer 5 [SIMPLE_EDGE]
@@ -94,7 +150,7 @@ Stage-0
                                         Select Operator [SEL_8] (rows=8116 width=1119)
                                           Output:["_col0"]
                                           Filter Operator [FIL_54] (rows=8116 width=1119)
-                                            predicate:((d_year) IN (1998, 1999, 2000) and (d_dom BETWEEN 1 AND 3 or d_dom BETWEEN 25 AND 28) and d_date_sk is not null)
+                                            predicate:((d_year) IN (2000, 2001, 2002) and (d_dom BETWEEN 1 AND 3 or d_dom BETWEEN 25 AND 28) 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_dom"]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query35.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query35.q.out b/ql/src/test/results/clientpositive/perf/query35.q.out
new file mode 100644
index 0000000..b286a07
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query35.q.out
@@ -0,0 +1,292 @@
+PREHOOK: query: explain
+select   
+  ca_state,
+  cd_gender,
+  cd_marital_status,
+  count(*) cnt1,
+  avg(cd_dep_count),
+  max(cd_dep_count),
+  sum(cd_dep_count),
+  cd_dep_employed_count,
+  count(*) cnt2,
+  avg(cd_dep_employed_count),
+  max(cd_dep_employed_count),
+  sum(cd_dep_employed_count),
+  cd_dep_college_count,
+  count(*) cnt3,
+  avg(cd_dep_college_count),
+  max(cd_dep_college_count),
+  sum(cd_dep_college_count)
+ from
+  customer c,customer_address ca,customer_demographics
+ where
+  c.c_current_addr_sk = ca.ca_address_sk and
+  cd_demo_sk = c.c_current_cdemo_sk and 
+  exists (select *
+          from store_sales,date_dim
+          where c.c_customer_sk = ss_customer_sk and
+                ss_sold_date_sk = d_date_sk and
+                d_year = 1999 and
+                d_qoy < 4) and
+   (exists (select *
+            from web_sales,date_dim
+            where c.c_customer_sk = ws_bill_customer_sk and
+                  ws_sold_date_sk = d_date_sk and
+                  d_year = 1999 and
+                  d_qoy < 4) or 
+    exists (select * 
+            from catalog_sales,date_dim
+            where c.c_customer_sk = cs_ship_customer_sk and
+                  cs_sold_date_sk = d_date_sk and
+                  d_year = 1999 and
+                  d_qoy < 4))
+ group by ca_state,
+          cd_gender,
+          cd_marital_status,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ order by ca_state,
+          cd_gender,
+          cd_marital_status,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select   
+  ca_state,
+  cd_gender,
+  cd_marital_status,
+  count(*) cnt1,
+  avg(cd_dep_count),
+  max(cd_dep_count),
+  sum(cd_dep_count),
+  cd_dep_employed_count,
+  count(*) cnt2,
+  avg(cd_dep_employed_count),
+  max(cd_dep_employed_count),
+  sum(cd_dep_employed_count),
+  cd_dep_college_count,
+  count(*) cnt3,
+  avg(cd_dep_college_count),
+  max(cd_dep_college_count),
+  sum(cd_dep_college_count)
+ from
+  customer c,customer_address ca,customer_demographics
+ where
+  c.c_current_addr_sk = ca.ca_address_sk and
+  cd_demo_sk = c.c_current_cdemo_sk and 
+  exists (select *
+          from store_sales,date_dim
+          where c.c_customer_sk = ss_customer_sk and
+                ss_sold_date_sk = d_date_sk and
+                d_year = 1999 and
+                d_qoy < 4) and
+   (exists (select *
+            from web_sales,date_dim
+            where c.c_customer_sk = ws_bill_customer_sk and
+                  ws_sold_date_sk = d_date_sk and
+                  d_year = 1999 and
+                  d_qoy < 4) or 
+    exists (select * 
+            from catalog_sales,date_dim
+            where c.c_customer_sk = cs_ship_customer_sk and
+                  cs_sold_date_sk = d_date_sk and
+                  d_year = 1999 and
+                  d_qoy < 4))
+ group by ca_state,
+          cd_gender,
+          cd_marital_status,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ order by ca_state,
+          cd_gender,
+          cd_marital_status,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 11 <- Map 10 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE)
+Reducer 12 <- Reducer 11 (SIMPLE_EDGE)
+Reducer 14 <- Map 13 (SIMPLE_EDGE), Map 18 (SIMPLE_EDGE)
+Reducer 15 <- Reducer 14 (SIMPLE_EDGE)
+Reducer 16 <- Map 13 (SIMPLE_EDGE), Map 19 (SIMPLE_EDGE)
+Reducer 17 <- Reducer 16 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 3 <- Map 9 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 15 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 17 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 7
+      File Output Operator [FS_75]
+        Limit [LIM_73] (rows=100 width=88)
+          Number of rows:100
+          Select Operator [SEL_72] (rows=383325119 width=88)
+            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16"]
+          <-Reducer 6 [SIMPLE_EDGE]
+            SHUFFLE [RS_71]
+              Select Operator [SEL_70] (rows=383325119 width=88)
+                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col9","_col10","_col11","_col12","_col14","_col15","_col16","_col17"]
+                Group By Operator [GBY_69] (rows=383325119 width=88)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15"],aggregations:["count(VALUE._col0)","avg(VALUE._col1)","max(VALUE._col2)","sum(VALUE._col3)","avg(VALUE._col4)","max(VALUE._col5)","sum(VALUE._col6)","avg(VALUE._col7)","max(VALUE._col8)","sum(VALUE._col9)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5
+                <-Reducer 5 [SIMPLE_EDGE]
+                  SHUFFLE [RS_68]
+                    PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5
+                    Group By Operator [GBY_67] (rows=766650239 width=88)
+                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15"],aggregations:["count()","avg(_col8)","max(_col8)","sum(_col8)","avg(_col9)","max(_col9)","sum(_col9)","avg(_col10)","max(_col10)","sum(_col10)"],keys:_col4, _col6, _col7, _col8, _col9, _col10
+                      Select Operator [SEL_66] (rows=766650239 width=88)
+                        Output:["_col4","_col6","_col7","_col8","_col9","_col10"]
+                        Filter Operator [FIL_65] (rows=766650239 width=88)
+                          predicate:(_col13 is not null or _col15 is not null)
+                          Merge Join Operator [MERGEJOIN_113] (rows=766650239 width=88)
+                            Conds:RS_62._col0=RS_63._col0(Left Outer),Output:["_col4","_col6","_col7","_col8","_col9","_col10","_col13","_col15"]
+                          <-Reducer 17 [SIMPLE_EDGE]
+                            SHUFFLE [RS_63]
+                              PartitionCols:_col0
+                              Select Operator [SEL_61] (rows=158394413 width=135)
+                                Output:["_col0","_col1"]
+                                Group By Operator [GBY_60] (rows=158394413 width=135)
+                                  Output:["_col0"],keys:KEY._col0
+                                <-Reducer 16 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_59]
+                                    PartitionCols:_col0
+                                    Group By Operator [GBY_58] (rows=316788826 width=135)
+                                      Output:["_col0"],keys:_col1
+                                      Merge Join Operator [MERGEJOIN_111] (rows=316788826 width=135)
+                                        Conds:RS_54._col0=RS_55._col0(Inner),Output:["_col1"]
+                                      <-Map 13 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_55]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_53] (rows=12174 width=1119)
+                                            Output:["_col0"]
+                                            Filter Operator [FIL_106] (rows=12174 width=1119)
+                                              predicate:((d_year = 1999) and (d_qoy < 4) and d_date_sk is not null)
+                                              TableScan [TS_12] (rows=73049 width=1119)
+                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"]
+                                      <-Map 19 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_54]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_50] (rows=287989836 width=135)
+                                            Output:["_col0","_col1"]
+                                            Filter Operator [FIL_105] (rows=287989836 width=135)
+                                              predicate:(cs_ship_customer_sk is not null and cs_sold_date_sk is not null)
+                                              TableScan [TS_48] (rows=287989836 width=135)
+                                                default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_ship_customer_sk"]
+                          <-Reducer 4 [SIMPLE_EDGE]
+                            SHUFFLE [RS_62]
+                              PartitionCols:_col0
+                              Select Operator [SEL_47] (rows=696954748 width=88)
+                                Output:["_col0","_col4","_col6","_col7","_col8","_col9","_col10","_col13"]
+                                Merge Join Operator [MERGEJOIN_112] (rows=696954748 width=88)
+                                  Conds:RS_43._col0=RS_44._col0(Left Outer),RS_43._col0=RS_45._col0(Inner),Output:["_col0","_col4","_col6","_col7","_col8","_col9","_col10","_col12"]
+                                <-Reducer 12 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_44]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_22] (rows=79201469 width=135)
+                                      Output:["_col0","_col1"]
+                                      Group By Operator [GBY_21] (rows=79201469 width=135)
+                                        Output:["_col0"],keys:KEY._col0
+                                      <-Reducer 11 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_20]
+                                          PartitionCols:_col0
+                                          Group By Operator [GBY_19] (rows=158402938 width=135)
+                                            Output:["_col0"],keys:_col1
+                                            Merge Join Operator [MERGEJOIN_109] (rows=158402938 width=135)
+                                              Conds:RS_15._col0=RS_16._col0(Inner),Output:["_col1"]
+                                            <-Map 13 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_16]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_14] (rows=12174 width=1119)
+                                                  Output:["_col0"]
+                                                  Filter Operator [FIL_102] (rows=12174 width=1119)
+                                                    predicate:((d_year = 1999) and (d_qoy < 4) and d_date_sk is not null)
+                                                     Please refer to the previous TableScan [TS_12]
+                                            <-Map 10 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_15]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_11] (rows=144002668 width=135)
+                                                  Output:["_col0","_col1"]
+                                                  Filter Operator [FIL_101] (rows=144002668 width=135)
+                                                    predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null)
+                                                    TableScan [TS_9] (rows=144002668 width=135)
+                                                      default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_bill_customer_sk"]
+                                <-Reducer 15 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_45]
+                                    PartitionCols:_col0
+                                    Group By Operator [GBY_35] (rows=316797606 width=88)
+                                      Output:["_col0"],keys:KEY._col0
+                                    <-Reducer 14 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_34]
+                                        PartitionCols:_col0
+                                        Group By Operator [GBY_33] (rows=633595212 width=88)
+                                          Output:["_col0"],keys:_col1
+                                          Merge Join Operator [MERGEJOIN_110] (rows=633595212 width=88)
+                                            Conds:RS_29._col0=RS_30._col0(Inner),Output:["_col1"]
+                                          <-Map 13 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_30]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_28] (rows=12174 width=1119)
+                                                Output:["_col0"]
+                                                Filter Operator [FIL_104] (rows=12174 width=1119)
+                                                  predicate:((d_year = 1999) and (d_qoy < 4) and d_date_sk is not null)
+                                                   Please refer to the previous TableScan [TS_12]
+                                          <-Map 18 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_29]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_25] (rows=575995635 width=88)
+                                                Output:["_col0","_col1"]
+                                                Filter Operator [FIL_103] (rows=575995635 width=88)
+                                                  predicate:(ss_customer_sk is not null and ss_sold_date_sk is not null)
+                                                  TableScan [TS_23] (rows=575995635 width=88)
+                                                    default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_customer_sk"]
+                                <-Reducer 3 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_43]
+                                    PartitionCols:_col0
+                                    Merge Join Operator [MERGEJOIN_108] (rows=96800003 width=860)
+                                      Conds:RS_40._col1=RS_41._col0(Inner),Output:["_col0","_col4","_col6","_col7","_col8","_col9","_col10"]
+                                    <-Map 9 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_41]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_8] (rows=1861800 width=385)
+                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                          Filter Operator [FIL_100] (rows=1861800 width=385)
+                                            predicate:cd_demo_sk is not null
+                                            TableScan [TS_6] (rows=1861800 width=385)
+                                              default@customer_demographics,customer_demographics,Tbl:COMPLETE,Col:NONE,Output:["cd_demo_sk","cd_gender","cd_marital_status","cd_dep_count","cd_dep_employed_count","cd_dep_college_count"]
+                                    <-Reducer 2 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_40]
+                                        PartitionCols:_col1
+                                        Merge Join Operator [MERGEJOIN_107] (rows=88000001 width=860)
+                                          Conds:RS_37._col2=RS_38._col0(Inner),Output:["_col0","_col1","_col4"]
+                                        <-Map 1 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_37]
+                                            PartitionCols:_col2
+                                            Select Operator [SEL_2] (rows=80000000 width=860)
+                                              Output:["_col0","_col1","_col2"]
+                                              Filter Operator [FIL_98] (rows=80000000 width=860)
+                                                predicate:(c_current_addr_sk is not null and c_current_cdemo_sk is not null)
+                                                TableScan [TS_0] (rows=80000000 width=860)
+                                                  default@customer,c,Tbl:COMPLETE,Col:NONE,Output:["c_customer_sk","c_current_cdemo_sk","c_current_addr_sk"]
+                                        <-Map 8 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_38]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_5] (rows=40000000 width=1014)
+                                              Output:["_col0","_col1"]
+                                              Filter Operator [FIL_99] (rows=40000000 width=1014)
+                                                predicate:ca_address_sk is not null
+                                                TableScan [TS_3] (rows=40000000 width=1014)
+                                                  default@customer_address,ca,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_state"]
+

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query36.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query36.q.out b/ql/src/test/results/clientpositive/perf/query36.q.out
index b5110d9..e0dc8d9 100644
--- a/ql/src/test/results/clientpositive/perf/query36.q.out
+++ b/ql/src/test/results/clientpositive/perf/query36.q.out
@@ -5,9 +5,9 @@ select
    ,i_class
    ,grouping(i_category)+grouping(i_class) as lochierarchy
    ,rank() over (
-     partition by grouping(i_category)+grouping(i_class),
-     case when grouping(i_class) = 0 then i_category end 
-     order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent
+ 	partition by grouping(i_category)+grouping(i_class),
+ 	case when grouping(i_class) = 0 then i_category end 
+ 	order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent
  from
     store_sales
    ,date_dim       d1
@@ -34,9 +34,9 @@ select
    ,i_class
    ,grouping(i_category)+grouping(i_class) as lochierarchy
    ,rank() over (
-     partition by grouping(i_category)+grouping(i_class),
-     case when grouping(i_class) = 0 then i_category end 
-     order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent
+ 	partition by grouping(i_category)+grouping(i_class),
+ 	case when grouping(i_class) = 0 then i_category end 
+ 	order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent
  from
     store_sales
    ,date_dim       d1

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query39.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query39.q.out b/ql/src/test/results/clientpositive/perf/query39.q.out
index 3e34c27..cf139f2 100644
--- a/ql/src/test/results/clientpositive/perf/query39.q.out
+++ b/ql/src/test/results/clientpositive/perf/query39.q.out
@@ -1,6 +1,54 @@
-PREHOOK: query: explain with inv as (select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stdev,mean, case mean when 0 then null else stdev/mean end cov from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean from inventory ,item ,warehouse ,date_dim where inv_item_sk = i_item_sk and inv_warehouse_sk = w_warehouse_sk and inv_date_sk = d_date_sk and d_year =1999 group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo where case mean when 0 then 0 else stdev/mean end > 1) select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov from inv inv1,inv inv2 where inv1.i_item_sk = inv2.i_item_sk and inv1.w_warehouse_sk = inv2.w_warehouse_sk and inv1.d_moy=3 and inv2.d_moy=3+1 order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov ,inv2.d_moy,inv2.mean, inv2.cov
+PREHOOK: query: explain
+with inv as
+(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+       ,stdev,mean, case mean when 0 then null else stdev/mean end cov
+ from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+            ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean
+      from inventory
+          ,item
+          ,warehouse
+          ,date_dim
+      where inv_item_sk = i_item_sk
+        and inv_warehouse_sk = w_warehouse_sk
+        and inv_date_sk = d_date_sk
+        and d_year =1999
+      group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo
+ where case mean when 0 then 0 else stdev/mean end > 1)
+select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov
+        ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov
+from inv inv1,inv inv2
+where inv1.i_item_sk = inv2.i_item_sk
+  and inv1.w_warehouse_sk =  inv2.w_warehouse_sk
+  and inv1.d_moy=4
+  and inv2.d_moy=4+1
+order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov
+        ,inv2.d_moy,inv2.mean, inv2.cov
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with inv as (select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stdev,mean, case mean when 0 then null else stdev/mean end cov from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean from inventory ,item ,warehouse ,date_dim where inv_item_sk = i_item_sk and inv_warehouse_sk = w_warehouse_sk and inv_date_sk = d_date_sk and d_year =1999 group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo where case mean when 0 then 0 else stdev/mean end > 1) select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov from inv inv1,inv inv2 where inv1.i_item_sk = inv2.i_item_sk and inv1.w_warehouse_sk = inv2.w_warehouse_sk and inv1.d_moy=3 and inv2.d_moy=3+1 order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov ,inv2.d_moy,inv2.mean, inv2.cov
+POSTHOOK: query: explain
+with inv as
+(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+       ,stdev,mean, case mean when 0 then null else stdev/mean end cov
+ from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+            ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean
+      from inventory
+          ,item
+          ,warehouse
+          ,date_dim
+      where inv_item_sk = i_item_sk
+        and inv_warehouse_sk = w_warehouse_sk
+        and inv_date_sk = d_date_sk
+        and d_year =1999
+      group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo
+ where case mean when 0 then 0 else stdev/mean end > 1)
+select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov
+        ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov
+from inv inv1,inv inv2
+where inv1.i_item_sk = inv2.i_item_sk
+  and inv1.w_warehouse_sk =  inv2.w_warehouse_sk
+  and inv1.d_moy=4
+  and inv2.d_moy=4+1
+order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov
+        ,inv2.d_moy,inv2.mean, inv2.cov
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -91,7 +139,7 @@ Stage-0
                                           Select Operator [SEL_33] (rows=18262 width=1119)
                                             Output:["_col0"]
                                             Filter Operator [FIL_95] (rows=18262 width=1119)
-                                              predicate:((d_year = 1999) and (d_moy = 4) and d_date_sk is not null)
+                                              predicate:((d_year = 1999) and (d_moy = 5) and d_date_sk is not null)
                                               TableScan [TS_3] (rows=73049 width=1119)
                                                 default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
               <-Reducer 5 [SIMPLE_EDGE]
@@ -152,17 +200,65 @@ Stage-0
                                           Select Operator [SEL_5] (rows=18262 width=1119)
                                             Output:["_col0"]
                                             Filter Operator [FIL_91] (rows=18262 width=1119)
-                                              predicate:((d_year = 1999) and (d_moy = 3) and d_date_sk is not null)
+                                              predicate:((d_year = 1999) and (d_moy = 4) and d_date_sk is not null)
                                                Please refer to the previous TableScan [TS_3]
 
-PREHOOK: query: with inv as (select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stdev,mean, case mean when 0 then null else stdev/mean end cov from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean from inventory ,item ,warehouse ,date_dim where inv_item_sk = i_item_sk and inv_warehouse_sk = w_warehouse_sk and inv_date_sk = d_date_sk and d_year =1999 group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo where case mean when 0 then 0 else stdev/mean end > 1) select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov from inv inv1,inv inv2 where inv1.i_item_sk = inv2.i_item_sk and inv1.w_warehouse_sk = inv2.w_warehouse_sk and inv1.d_moy=3 and inv2.d_moy=3+1 and inv1.cov > 1.5 order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov ,inv2.d_moy,inv2.mean, inv2.cov
+PREHOOK: query: with inv as
+(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+       ,stdev,mean, case mean when 0 then null else stdev/mean end cov
+ from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+            ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean
+      from inventory
+          ,item
+          ,warehouse
+          ,date_dim
+      where inv_item_sk = i_item_sk
+        and inv_warehouse_sk = w_warehouse_sk
+        and inv_date_sk = d_date_sk
+        and d_year =1999
+      group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo
+ where case mean when 0 then 0 else stdev/mean end > 1)
+select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov
+        ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov
+from inv inv1,inv inv2
+where inv1.i_item_sk = inv2.i_item_sk
+  and inv1.w_warehouse_sk =  inv2.w_warehouse_sk
+  and inv1.d_moy=4
+  and inv2.d_moy=4+1
+  and inv1.cov > 1.5
+order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov
+        ,inv2.d_moy,inv2.mean, inv2.cov
 PREHOOK: type: QUERY
 PREHOOK: Input: default@date_dim
 PREHOOK: Input: default@inventory
 PREHOOK: Input: default@item
 PREHOOK: Input: default@warehouse
 #### A masked pattern was here ####
-POSTHOOK: query: with inv as (select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stdev,mean, case mean when 0 then null else stdev/mean end cov from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean from inventory ,item ,warehouse ,date_dim where inv_item_sk = i_item_sk and inv_warehouse_sk = w_warehouse_sk and inv_date_sk = d_date_sk and d_year =1999 group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo where case mean when 0 then 0 else stdev/mean end > 1) select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov from inv inv1,inv inv2 where inv1.i_item_sk = inv2.i_item_sk and inv1.w_warehouse_sk = inv2.w_warehouse_sk and inv1.d_moy=3 and inv2.d_moy=3+1 and inv1.cov > 1.5 order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov ,inv2.d_moy,inv2.mean, inv2.cov
+POSTHOOK: query: with inv as
+(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+       ,stdev,mean, case mean when 0 then null else stdev/mean end cov
+ from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+            ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean
+      from inventory
+          ,item
+          ,warehouse
+          ,date_dim
+      where inv_item_sk = i_item_sk
+        and inv_warehouse_sk = w_warehouse_sk
+        and inv_date_sk = d_date_sk
+        and d_year =1999
+      group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo
+ where case mean when 0 then 0 else stdev/mean end > 1)
+select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov
+        ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov
+from inv inv1,inv inv2
+where inv1.i_item_sk = inv2.i_item_sk
+  and inv1.w_warehouse_sk =  inv2.w_warehouse_sk
+  and inv1.d_moy=4
+  and inv2.d_moy=4+1
+  and inv1.cov > 1.5
+order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov
+        ,inv2.d_moy,inv2.mean, inv2.cov
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@date_dim
 POSTHOOK: Input: default@inventory

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query42.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query42.q.out b/ql/src/test/results/clientpositive/perf/query42.q.out
index 307ef98..1ccb4fd 100644
--- a/ql/src/test/results/clientpositive/perf/query42.q.out
+++ b/ql/src/test/results/clientpositive/perf/query42.q.out
@@ -1,6 +1,44 @@
-PREHOOK: query: explain select dt.d_year ,item.i_category_id ,item.i_category ,sum(ss_ext_sales_price) as s from date_dim dt ,store_sales ,item where dt.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and item.i_manager_id = 1 and dt.d_moy=12 and dt.d_year=1998 group by dt.d_year ,item.i_category_id ,item.i_category order by s desc,dt.d_year ,item.i_category_id ,item.i_category limit 100
+PREHOOK: query: explain
+select  dt.d_year
+ 	,item.i_category_id
+ 	,item.i_category
+ 	,sum(ss_ext_sales_price)
+ from 	date_dim dt
+ 	,store_sales
+ 	,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+ 	and store_sales.ss_item_sk = item.i_item_sk
+ 	and item.i_manager_id = 1  	
+ 	and dt.d_moy=12
+ 	and dt.d_year=1998
+ group by 	dt.d_year
+ 		,item.i_category_id
+ 		,item.i_category
+ order by       sum(ss_ext_sales_price) desc,dt.d_year
+ 		,item.i_category_id
+ 		,item.i_category
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select dt.d_year ,item.i_category_id ,item.i_category ,sum(ss_ext_sales_price) as s from date_dim dt ,store_sales ,item where dt.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and item.i_manager_id = 1 and dt.d_moy=12 and dt.d_year=1998 group by dt.d_year ,item.i_category_id ,item.i_category order by s desc,dt.d_year ,item.i_category_id ,item.i_category limit 100
+POSTHOOK: query: explain
+select  dt.d_year
+ 	,item.i_category_id
+ 	,item.i_category
+ 	,sum(ss_ext_sales_price)
+ from 	date_dim dt
+ 	,store_sales
+ 	,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+ 	and store_sales.ss_item_sk = item.i_item_sk
+ 	and item.i_manager_id = 1  	
+ 	and dt.d_moy=12
+ 	and dt.d_year=1998
+ group by 	dt.d_year
+ 		,item.i_category_id
+ 		,item.i_category
+ order by       sum(ss_ext_sales_price) desc,dt.d_year
+ 		,item.i_category_id
+ 		,item.i_category
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -24,45 +62,47 @@ Stage-0
               Output:["_col0","_col1","_col2"]
             <-Reducer 4 [SIMPLE_EDGE]
               SHUFFLE [RS_20]
-                Group By Operator [GBY_18] (rows=348477374 width=88)
-                  Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
-                <-Reducer 3 [SIMPLE_EDGE]
-                  SHUFFLE [RS_17]
-                    PartitionCols:_col0, _col1
-                    Group By Operator [GBY_16] (rows=696954748 width=88)
-                      Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)"],keys:_col7, _col8
-                      Merge Join Operator [MERGEJOIN_34] (rows=696954748 width=88)
-                        Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col2","_col7","_col8"]
-                      <-Map 7 [SIMPLE_EDGE]
-                        SHUFFLE [RS_13]
-                          PartitionCols:_col0
-                          Select Operator [SEL_8] (rows=231000 width=1436)
-                            Output:["_col0","_col1","_col2"]
-                            Filter Operator [FIL_32] (rows=231000 width=1436)
-                              predicate:((i_manager_id = 1) and i_item_sk is not null)
-                              TableScan [TS_6] (rows=462000 width=1436)
-                                default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_category_id","i_category","i_manager_id"]
-                      <-Reducer 2 [SIMPLE_EDGE]
-                        SHUFFLE [RS_12]
-                          PartitionCols:_col1
-                          Merge Join Operator [MERGEJOIN_33] (rows=633595212 width=88)
-                            Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col1","_col2"]
-                          <-Map 1 [SIMPLE_EDGE]
-                            SHUFFLE [RS_9]
-                              PartitionCols:_col0
-                              Select Operator [SEL_2] (rows=575995635 width=88)
-                                Output:["_col0","_col1","_col2"]
-                                Filter Operator [FIL_30] (rows=575995635 width=88)
-                                  predicate:(ss_sold_date_sk is not null and ss_item_sk is not null)
-                                  TableScan [TS_0] (rows=575995635 width=88)
-                                    default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ext_sales_price"]
-                          <-Map 6 [SIMPLE_EDGE]
-                            SHUFFLE [RS_10]
-                              PartitionCols:_col0
-                              Select Operator [SEL_5] (rows=18262 width=1119)
-                                Output:["_col0"]
-                                Filter Operator [FIL_31] (rows=18262 width=1119)
-                                  predicate:((d_moy = 12) and (d_year = 1998) and d_date_sk is not null)
-                                  TableScan [TS_3] (rows=73049 width=1119)
-                                    default@date_dim,dt,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
+                Select Operator [SEL_19] (rows=348477374 width=88)
+                  Output:["_col0","_col1","_col3"]
+                  Group By Operator [GBY_18] (rows=348477374 width=88)
+                    Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
+                  <-Reducer 3 [SIMPLE_EDGE]
+                    SHUFFLE [RS_17]
+                      PartitionCols:_col0, _col1
+                      Group By Operator [GBY_16] (rows=696954748 width=88)
+                        Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)"],keys:_col7, _col8
+                        Merge Join Operator [MERGEJOIN_34] (rows=696954748 width=88)
+                          Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col2","_col7","_col8"]
+                        <-Map 7 [SIMPLE_EDGE]
+                          SHUFFLE [RS_13]
+                            PartitionCols:_col0
+                            Select Operator [SEL_8] (rows=231000 width=1436)
+                              Output:["_col0","_col1","_col2"]
+                              Filter Operator [FIL_32] (rows=231000 width=1436)
+                                predicate:((i_manager_id = 1) and i_item_sk is not null)
+                                TableScan [TS_6] (rows=462000 width=1436)
+                                  default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_category_id","i_category","i_manager_id"]
+                        <-Reducer 2 [SIMPLE_EDGE]
+                          SHUFFLE [RS_12]
+                            PartitionCols:_col1
+                            Merge Join Operator [MERGEJOIN_33] (rows=633595212 width=88)
+                              Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col1","_col2"]
+                            <-Map 1 [SIMPLE_EDGE]
+                              SHUFFLE [RS_9]
+                                PartitionCols:_col0
+                                Select Operator [SEL_2] (rows=575995635 width=88)
+                                  Output:["_col0","_col1","_col2"]
+                                  Filter Operator [FIL_30] (rows=575995635 width=88)
+                                    predicate:(ss_sold_date_sk is not null and ss_item_sk is not null)
+                                    TableScan [TS_0] (rows=575995635 width=88)
+                                      default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ext_sales_price"]
+                            <-Map 6 [SIMPLE_EDGE]
+                              SHUFFLE [RS_10]
+                                PartitionCols:_col0
+                                Select Operator [SEL_5] (rows=18262 width=1119)
+                                  Output:["_col0"]
+                                  Filter Operator [FIL_31] (rows=18262 width=1119)
+                                    predicate:((d_moy = 12) and (d_year = 1998) and d_date_sk is not null)
+                                    TableScan [TS_3] (rows=73049 width=1119)
+                                      default@date_dim,dt,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query43.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query43.q.out b/ql/src/test/results/clientpositive/perf/query43.q.out
index 72712e8..6db0919 100644
--- a/ql/src/test/results/clientpositive/perf/query43.q.out
+++ b/ql/src/test/results/clientpositive/perf/query43.q.out
@@ -1,6 +1,38 @@
-PREHOOK: query: explain select s_store_name, s_store_id, sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales from date_dim, store_sales, store where date_dim.d_date_sk = store_sales.ss_sold_date_sk and store.s_store_sk = store_sales.ss_store_sk and s_gmt_offset = -6 and d_year = 1998 group by s_store_name, s_store_id order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales limit 100
+PREHOOK: query: explain
+select  s_store_name, s_store_id,
+        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from date_dim, store_sales, store
+ where d_date_sk = ss_sold_date_sk and
+       s_store_sk = ss_store_sk and
+       s_gmt_offset = -6 and
+       d_year = 1998 
+ group by s_store_name, s_store_id
+ order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select s_store_name, s_store_id, sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales from date_dim, store_sales, store where date_dim.d_date_sk = store_sales.ss_sold_date_sk and store.s_store_sk = store_sales.ss_store_sk and s_gmt_offset = -6 and d_year = 1998 group by s_store_name, s_store_id order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales limit 100
+POSTHOOK: query: explain
+select  s_store_name, s_store_id,
+        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from date_dim, store_sales, store
+ where d_date_sk = ss_sold_date_sk and
+       s_store_sk = ss_store_sk and
+       s_gmt_offset = -6 and
+       d_year = 1998 
+ group by s_store_name, s_store_id
+ order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query44.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query44.q.out b/ql/src/test/results/clientpositive/perf/query44.q.out
new file mode 100644
index 0000000..5665480
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query44.q.out
@@ -0,0 +1,279 @@
+Warning: Shuffle Join MERGEJOIN[131][tables = [$hdt$_2, $hdt$_3, $hdt$_1]] in Stage 'Reducer 9' is a cross product
+Warning: Shuffle Join MERGEJOIN[133][tables = [$hdt$_4, $hdt$_5, $hdt$_3]] in Stage 'Reducer 13' is a cross product
+PREHOOK: query: explain
+select  asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing
+from(select *
+     from (select item_sk,rank() over (order by rank_col asc) rnk
+           from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col 
+                 from store_sales ss1
+                 where ss_store_sk = 410
+                 group by ss_item_sk
+                 having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col
+                                                  from store_sales
+                                                  where ss_store_sk = 410
+                                                    and ss_hdemo_sk is null
+                                                  group by ss_store_sk))V1)V11
+     where rnk  < 11) asceding,
+    (select *
+     from (select item_sk,rank() over (order by rank_col desc) rnk
+           from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col
+                 from store_sales ss1
+                 where ss_store_sk = 410
+                 group by ss_item_sk
+                 having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col
+                                                  from store_sales
+                                                  where ss_store_sk = 410
+                                                    and ss_hdemo_sk is null
+                                                  group by ss_store_sk))V2)V21
+     where rnk  < 11) descending,
+item i1,
+item i2
+where asceding.rnk = descending.rnk 
+  and i1.i_item_sk=asceding.item_sk
+  and i2.i_item_sk=descending.item_sk
+order by asceding.rnk
+limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select  asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing
+from(select *
+     from (select item_sk,rank() over (order by rank_col asc) rnk
+           from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col 
+                 from store_sales ss1
+                 where ss_store_sk = 410
+                 group by ss_item_sk
+                 having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col
+                                                  from store_sales
+                                                  where ss_store_sk = 410
+                                                    and ss_hdemo_sk is null
+                                                  group by ss_store_sk))V1)V11
+     where rnk  < 11) asceding,
+    (select *
+     from (select item_sk,rank() over (order by rank_col desc) rnk
+           from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col
+                 from store_sales ss1
+                 where ss_store_sk = 410
+                 group by ss_item_sk
+                 having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col
+                                                  from store_sales
+                                                  where ss_store_sk = 410
+                                                    and ss_hdemo_sk is null
+                                                  group by ss_store_sk))V2)V21
+     where rnk  < 11) descending,
+item i1,
+item i2
+where asceding.rnk = descending.rnk 
+  and i1.i_item_sk=asceding.item_sk
+  and i2.i_item_sk=descending.item_sk
+order by asceding.rnk
+limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 10 <- Reducer 9 (SIMPLE_EDGE)
+Reducer 11 <- Map 6 (SIMPLE_EDGE)
+Reducer 12 <- Reducer 11 (CUSTOM_SIMPLE_EDGE)
+Reducer 13 <- Reducer 12 (CUSTOM_SIMPLE_EDGE), Reducer 17 (CUSTOM_SIMPLE_EDGE), Reducer 20 (CUSTOM_SIMPLE_EDGE)
+Reducer 14 <- Reducer 13 (SIMPLE_EDGE)
+Reducer 16 <- Map 15 (SIMPLE_EDGE)
+Reducer 17 <- Map 15 (SIMPLE_EDGE)
+Reducer 19 <- Map 18 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE)
+Reducer 20 <- Map 18 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Map 1 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE)
+Reducer 7 <- Map 6 (SIMPLE_EDGE)
+Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
+Reducer 9 <- Reducer 16 (CUSTOM_SIMPLE_EDGE), Reducer 19 (CUSTOM_SIMPLE_EDGE), Reducer 8 (CUSTOM_SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 4
+      File Output Operator [FS_104]
+        Limit [LIM_103] (rows=100 width=185)
+          Number of rows:100
+          Select Operator [SEL_102] (rows=1393898919384048 width=185)
+            Output:["_col0","_col1","_col2"]
+          <-Reducer 3 [SIMPLE_EDGE]
+            SHUFFLE [RS_101]
+              Select Operator [SEL_100] (rows=1393898919384048 width=185)
+                Output:["_col0","_col1","_col2"]
+                Merge Join Operator [MERGEJOIN_135] (rows=1393898919384048 width=185)
+                  Conds:RS_97._col3=RS_98._col3(Inner),Output:["_col1","_col3","_col5"]
+                <-Reducer 2 [SIMPLE_EDGE]
+                  SHUFFLE [RS_97]
+                    PartitionCols:_col3
+                    Merge Join Operator [MERGEJOIN_132] (rows=1267180808338276 width=185)
+                      Conds:RS_94._col0=RS_95._col0(Inner),Output:["_col1","_col3"]
+                    <-Map 1 [SIMPLE_EDGE]
+                      SHUFFLE [RS_94]
+                        PartitionCols:_col0
+                        Select Operator [SEL_2] (rows=462000 width=1436)
+                          Output:["_col0","_col1"]
+                          Filter Operator [FIL_121] (rows=462000 width=1436)
+                            predicate:i_item_sk is not null
+                            TableScan [TS_0] (rows=462000 width=1436)
+                              default@item,i1,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_product_name"]
+                    <-Reducer 10 [SIMPLE_EDGE]
+                      SHUFFLE [RS_95]
+                        PartitionCols:_col0
+                        Select Operator [SEL_41] (rows=1151982528066248 width=185)
+                          Output:["_col0","_col1"]
+                          Filter Operator [FIL_122] (rows=1151982528066248 width=185)
+                            predicate:((rank_window_0 < 11) and _col2 is not null)
+                            PTF Operator [PTF_40] (rows=3455947584198744 width=185)
+                              Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3 ASC NULLS FIRST","partition by:":"0"}]
+                              Select Operator [SEL_39] (rows=3455947584198744 width=185)
+                                Output:["_col2","_col3"]
+                              <-Reducer 9 [SIMPLE_EDGE]
+                                SHUFFLE [RS_38]
+                                  PartitionCols:0
+                                  Filter Operator [FIL_37] (rows=3455947584198744 width=185)
+                                    predicate:(_col3 > (0.9 * _col1))
+                                    Merge Join Operator [MERGEJOIN_131] (rows=10367842752596232 width=185)
+                                      Conds:(Inner),(Inner),Output:["_col1","_col2","_col3"]
+                                    <-Reducer 16 [CUSTOM_SIMPLE_EDGE]
+                                      PARTITION_ONLY_SHUFFLE [RS_34]
+                                        Select Operator [SEL_25] (rows=71999454 width=88)
+                                          Output:["_col0"]
+                                          Group By Operator [GBY_24] (rows=71999454 width=88)
+                                            Output:["_col0","_col1"],aggregations:["avg(VALUE._col0)"],keys:KEY._col0
+                                          <-Map 15 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_23]
+                                              PartitionCols:_col0
+                                              Group By Operator [GBY_22] (rows=143998908 width=88)
+                                                Output:["_col0","_col1"],aggregations:["avg(_col1)"],keys:410
+                                                Select Operator [SEL_20] (rows=143998908 width=88)
+                                                  Output:["_col1"]
+                                                  Filter Operator [FIL_124] (rows=143998908 width=88)
+                                                    predicate:((ss_store_sk = 410) and ss_hdemo_sk is null)
+                                                    TableScan [TS_18] (rows=575995635 width=88)
+                                                      default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_hdemo_sk","ss_store_sk","ss_net_profit"]
+                                    <-Reducer 19 [CUSTOM_SIMPLE_EDGE]
+                                      PARTITION_ONLY_SHUFFLE [RS_35]
+                                        Group By Operator [GBY_31] (rows=143998908 width=88)
+                                          Output:["_col0","_col1"],aggregations:["avg(VALUE._col0)"],keys:KEY._col0
+                                        <-Map 18 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_30]
+                                            PartitionCols:_col0
+                                            Group By Operator [GBY_29] (rows=287997817 width=88)
+                                              Output:["_col0","_col1"],aggregations:["avg(ss_net_profit)"],keys:ss_item_sk
+                                              Select Operator [SEL_28] (rows=287997817 width=88)
+                                                Output:["ss_item_sk","ss_net_profit"]
+                                                Filter Operator [FIL_125] (rows=287997817 width=88)
+                                                  predicate:(ss_store_sk = 410)
+                                                  TableScan [TS_26] (rows=575995635 width=88)
+                                                    default@store_sales,ss1,Tbl:COMPLETE,Col:NONE,Output:["ss_item_sk","ss_store_sk","ss_net_profit"]
+                                    <-Reducer 8 [CUSTOM_SIMPLE_EDGE]
+                                      PARTITION_ONLY_SHUFFLE [RS_33]
+                                        Select Operator [SEL_17] (rows=1 width=8)
+                                          Filter Operator [FIL_16] (rows=1 width=8)
+                                            predicate:(sq_count_check(_col0) <= 1)
+                                            Group By Operator [GBY_14] (rows=1 width=8)
+                                              Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                                            <-Reducer 7 [CUSTOM_SIMPLE_EDGE]
+                                              PARTITION_ONLY_SHUFFLE [RS_13]
+                                                Group By Operator [GBY_12] (rows=1 width=8)
+                                                  Output:["_col0"],aggregations:["count()"]
+                                                  Select Operator [SEL_10] (rows=71999454 width=88)
+                                                    Group By Operator [GBY_9] (rows=71999454 width=88)
+                                                      Output:["_col0"],keys:KEY._col0
+                                                    <-Map 6 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_8]
+                                                        PartitionCols:_col0
+                                                        Group By Operator [GBY_7] (rows=143998908 width=88)
+                                                          Output:["_col0"],keys:410
+                                                          Select Operator [SEL_5] (rows=143998908 width=88)
+                                                            Filter Operator [FIL_123] (rows=143998908 width=88)
+                                                              predicate:((ss_store_sk = 410) and ss_hdemo_sk is null)
+                                                              TableScan [TS_3] (rows=575995635 width=88)
+                                                                default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_hdemo_sk","ss_store_sk"]
+                <-Reducer 5 [SIMPLE_EDGE]
+                  SHUFFLE [RS_98]
+                    PartitionCols:_col3
+                    Merge Join Operator [MERGEJOIN_134] (rows=1267180808338276 width=185)
+                      Conds:RS_90._col0=RS_91._col0(Inner),Output:["_col1","_col3"]
+                    <-Map 1 [SIMPLE_EDGE]
+                      SHUFFLE [RS_90]
+                        PartitionCols:_col0
+                        Select Operator [SEL_47] (rows=462000 width=1436)
+                          Output:["_col0","_col1"]
+                          Filter Operator [FIL_126] (rows=462000 width=1436)
+                            predicate:i_item_sk is not null
+                             Please refer to the previous TableScan [TS_0]
+                    <-Reducer 14 [SIMPLE_EDGE]
+                      SHUFFLE [RS_91]
+                        PartitionCols:_col0
+                        Select Operator [SEL_86] (rows=1151982528066248 width=185)
+                          Output:["_col0","_col1"]
+                          Filter Operator [FIL_127] (rows=1151982528066248 width=185)
+                            predicate:((rank_window_0 < 11) and _col2 is not null)
+                            PTF Operator [PTF_85] (rows=3455947584198744 width=185)
+                              Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3 DESC NULLS LAST","partition by:":"0"}]
+                              Select Operator [SEL_84] (rows=3455947584198744 width=185)
+                                Output:["_col2","_col3"]
+                              <-Reducer 13 [SIMPLE_EDGE]
+                                SHUFFLE [RS_83]
+                                  PartitionCols:0
+                                  Filter Operator [FIL_82] (rows=3455947584198744 width=185)
+                                    predicate:(_col3 > (0.9 * _col1))
+                                    Merge Join Operator [MERGEJOIN_133] (rows=10367842752596232 width=185)
+                                      Conds:(Inner),(Inner),Output:["_col1","_col2","_col3"]
+                                    <-Reducer 12 [CUSTOM_SIMPLE_EDGE]
+                                      PARTITION_ONLY_SHUFFLE [RS_78]
+                                        Select Operator [SEL_62] (rows=1 width=8)
+                                          Filter Operator [FIL_61] (rows=1 width=8)
+                                            predicate:(sq_count_check(_col0) <= 1)
+                                            Group By Operator [GBY_59] (rows=1 width=8)
+                                              Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                                            <-Reducer 11 [CUSTOM_SIMPLE_EDGE]
+                                              PARTITION_ONLY_SHUFFLE [RS_58]
+                                                Group By Operator [GBY_57] (rows=1 width=8)
+                                                  Output:["_col0"],aggregations:["count()"]
+                                                  Select Operator [SEL_55] (rows=71999454 width=88)
+                                                    Group By Operator [GBY_54] (rows=71999454 width=88)
+                                                      Output:["_col0"],keys:KEY._col0
+                                                    <-Map 6 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_53]
+                                                        PartitionCols:_col0
+                                                        Group By Operator [GBY_52] (rows=143998908 width=88)
+                                                          Output:["_col0"],keys:410
+                                                          Select Operator [SEL_50] (rows=143998908 width=88)
+                                                            Filter Operator [FIL_128] (rows=143998908 width=88)
+                                                              predicate:((ss_store_sk = 410) and ss_hdemo_sk is null)
+                                                               Please refer to the previous TableScan [TS_3]
+                                    <-Reducer 17 [CUSTOM_SIMPLE_EDGE]
+                                      PARTITION_ONLY_SHUFFLE [RS_79]
+                                        Select Operator [SEL_70] (rows=71999454 width=88)
+                                          Output:["_col0"]
+                                          Group By Operator [GBY_69] (rows=71999454 width=88)
+                                            Output:["_col0","_col1"],aggregations:["avg(VALUE._col0)"],keys:KEY._col0
+                                          <-Map 15 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_68]
+                                              PartitionCols:_col0
+                                              Group By Operator [GBY_67] (rows=143998908 width=88)
+                                                Output:["_col0","_col1"],aggregations:["avg(_col1)"],keys:410
+                                                Select Operator [SEL_65] (rows=143998908 width=88)
+                                                  Output:["_col1"]
+                                                  Filter Operator [FIL_129] (rows=143998908 width=88)
+                                                    predicate:((ss_store_sk = 410) and ss_hdemo_sk is null)
+                                                     Please refer to the previous TableScan [TS_18]
+                                    <-Reducer 20 [CUSTOM_SIMPLE_EDGE]
+                                      PARTITION_ONLY_SHUFFLE [RS_80]
+                                        Group By Operator [GBY_76] (rows=143998908 width=88)
+                                          Output:["_col0","_col1"],aggregations:["avg(VALUE._col0)"],keys:KEY._col0
+                                        <-Map 18 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_75]
+                                            PartitionCols:_col0
+                                            Group By Operator [GBY_74] (rows=287997817 width=88)
+                                              Output:["_col0","_col1"],aggregations:["avg(ss_net_profit)"],keys:ss_item_sk
+                                              Select Operator [SEL_73] (rows=287997817 width=88)
+                                                Output:["ss_item_sk","ss_net_profit"]
+                                                Filter Operator [FIL_130] (rows=287997817 width=88)
+                                                  predicate:(ss_store_sk = 410)
+                                                   Please refer to the previous TableScan [TS_26]
+

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query45.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query45.q.out b/ql/src/test/results/clientpositive/perf/query45.q.out
new file mode 100644
index 0000000..19cf365
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query45.q.out
@@ -0,0 +1,180 @@
+Warning: Shuffle Join MERGEJOIN[87][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product
+PREHOOK: query: explain
+select  ca_zip, ca_county, sum(ws_sales_price)
+ from web_sales, customer, customer_address, date_dim, item
+ where ws_bill_customer_sk = c_customer_sk
+ 	and c_current_addr_sk = ca_address_sk 
+ 	and ws_item_sk = i_item_sk 
+ 	and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792')
+ 	      or 
+ 	      i_item_id in (select i_item_id
+                             from item
+                             where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)
+                             )
+ 	    )
+ 	and ws_sold_date_sk = d_date_sk
+ 	and d_qoy = 2 and d_year = 2000
+ group by ca_zip, ca_county
+ order by ca_zip, ca_county
+ limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select  ca_zip, ca_county, sum(ws_sales_price)
+ from web_sales, customer, customer_address, date_dim, item
+ where ws_bill_customer_sk = c_customer_sk
+ 	and c_current_addr_sk = ca_address_sk 
+ 	and ws_item_sk = i_item_sk 
+ 	and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792')
+ 	      or 
+ 	      i_item_id in (select i_item_id
+                             from item
+                             where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)
+                             )
+ 	    )
+ 	and ws_sold_date_sk = d_date_sk
+ 	and d_qoy = 2 and d_year = 2000
+ group by ca_zip, ca_county
+ order by ca_zip, ca_county
+ limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 10 <- Reducer 14 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+Reducer 11 <- Map 8 (SIMPLE_EDGE)
+Reducer 12 <- Map 8 (CUSTOM_SIMPLE_EDGE)
+Reducer 14 <- Map 13 (SIMPLE_EDGE), Map 15 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 10 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 12 (CUSTOM_SIMPLE_EDGE), Reducer 3 (CUSTOM_SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 9 <- Map 8 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 6
+      File Output Operator [FS_59]
+        Limit [LIM_58] (rows=100 width=152)
+          Number of rows:100
+          Select Operator [SEL_57] (rows=95833781 width=152)
+            Output:["_col0","_col1","_col2"]
+          <-Reducer 5 [SIMPLE_EDGE]
+            SHUFFLE [RS_56]
+              Group By Operator [GBY_54] (rows=95833781 width=152)
+                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
+              <-Reducer 4 [SIMPLE_EDGE]
+                SHUFFLE [RS_53]
+                  PartitionCols:_col0, _col1
+                  Group By Operator [GBY_52] (rows=191667562 width=152)
+                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col8, _col7
+                    Select Operator [SEL_51] (rows=191667562 width=152)
+                      Output:["_col3","_col7","_col8"]
+                      Filter Operator [FIL_50] (rows=191667562 width=152)
+                        predicate:((substr(_col8, 1, 5)) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') or CASE WHEN ((_col14 = 0)) THEN (false) WHEN (_col17 is not null) THEN (true) WHEN (_col13 is null) THEN (null) WHEN ((_col15 < _col14)) THEN (null) ELSE (false) END)
+                        Select Operator [SEL_49] (rows=191667562 width=152)
+                          Output:["_col3","_col7","_col8","_col13","_col14","_col15","_col17"]
+                          Merge Join Operator [MERGEJOIN_87] (rows=191667562 width=152)
+                            Conds:(Inner),Output:["_col3","_col4","_col6","_col8","_col12","_col16","_col17"]
+                          <-Reducer 12 [CUSTOM_SIMPLE_EDGE]
+                            PARTITION_ONLY_SHUFFLE [RS_47]
+                              Group By Operator [GBY_38] (rows=1 width=16)
+                                Output:["_col0","_col1"],aggregations:["count(VALUE._col0)","count(VALUE._col1)"]
+                              <-Map 8 [CUSTOM_SIMPLE_EDGE]
+                                SHUFFLE [RS_37]
+                                  Group By Operator [GBY_36] (rows=1 width=16)
+                                    Output:["_col0","_col1"],aggregations:["count()","count(i_item_id)"]
+                                    Select Operator [SEL_35] (rows=231000 width=1436)
+                                      Output:["i_item_id"]
+                                      Filter Operator [FIL_81] (rows=231000 width=1436)
+                                        predicate:(i_item_sk) IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)
+                                        TableScan [TS_6] (rows=462000 width=1436)
+                                          default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_item_id"]
+                          <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
+                            PARTITION_ONLY_SHUFFLE [RS_46]
+                              Merge Join Operator [MERGEJOIN_86] (rows=191667562 width=135)
+                                Conds:RS_43._col0=RS_44._col6(Inner),Output:["_col3","_col4","_col6","_col8","_col12"]
+                              <-Reducer 10 [SIMPLE_EDGE]
+                                SHUFFLE [RS_44]
+                                  PartitionCols:_col6
+                                  Merge Join Operator [MERGEJOIN_85] (rows=174243235 width=135)
+                                    Conds:RS_29._col0=RS_30._col1(Inner),Output:["_col1","_col3","_col6","_col7"]
+                                  <-Reducer 14 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_30]
+                                      PartitionCols:_col1
+                                      Merge Join Operator [MERGEJOIN_84] (rows=158402938 width=135)
+                                        Conds:RS_22._col0=RS_23._col0(Inner),Output:["_col1","_col2","_col3"]
+                                      <-Map 13 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_22]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_18] (rows=144002668 width=135)
+                                            Output:["_col0","_col1","_col2","_col3"]
+                                            Filter Operator [FIL_79] (rows=144002668 width=135)
+                                              predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null and ws_item_sk is not null)
+                                              TableScan [TS_16] (rows=144002668 width=135)
+                                                default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_bill_customer_sk","ws_sales_price"]
+                                      <-Map 15 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_23]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_21] (rows=18262 width=1119)
+                                            Output:["_col0"]
+                                            Filter Operator [FIL_80] (rows=18262 width=1119)
+                                              predicate:((d_qoy = 2) and (d_year = 2000) and d_date_sk is not null)
+                                              TableScan [TS_19] (rows=73049 width=1119)
+                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"]
+                                  <-Reducer 9 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_29]
+                                      PartitionCols:_col0
+                                      Merge Join Operator [MERGEJOIN_83] (rows=508200 width=1436)
+                                        Conds:RS_26._col1=RS_27._col0(Left Outer),Output:["_col0","_col1","_col3"]
+                                      <-Map 8 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_26]
+                                          PartitionCols:_col1
+                                          Select Operator [SEL_8] (rows=462000 width=1436)
+                                            Output:["_col0","_col1"]
+                                            Filter Operator [FIL_77] (rows=462000 width=1436)
+                                              predicate:i_item_sk is not null
+                                               Please refer to the previous TableScan [TS_6]
+                                      <-Reducer 11 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_27]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_15] (rows=115500 width=1436)
+                                            Output:["_col0","_col1"]
+                                            Group By Operator [GBY_14] (rows=115500 width=1436)
+                                              Output:["_col0"],keys:KEY._col0
+                                            <-Map 8 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_13]
+                                                PartitionCols:_col0
+                                                Group By Operator [GBY_12] (rows=231000 width=1436)
+                                                  Output:["_col0"],keys:i_item_id
+                                                  Select Operator [SEL_11] (rows=231000 width=1436)
+                                                    Output:["i_item_id"]
+                                                    Filter Operator [FIL_78] (rows=231000 width=1436)
+                                                      predicate:(i_item_sk) IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)
+                                                       Please refer to the previous TableScan [TS_6]
+                              <-Reducer 2 [SIMPLE_EDGE]
+                                SHUFFLE [RS_43]
+                                  PartitionCols:_col0
+                                  Merge Join Operator [MERGEJOIN_82] (rows=88000001 width=860)
+                                    Conds:RS_40._col1=RS_41._col0(Inner),Output:["_col0","_col3","_col4"]
+                                  <-Map 1 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_40]
+                                      PartitionCols:_col1
+                                      Select Operator [SEL_2] (rows=80000000 width=860)
+                                        Output:["_col0","_col1"]
+                                        Filter Operator [FIL_75] (rows=80000000 width=860)
+                                          predicate:(c_customer_sk is not null and c_current_addr_sk is not null)
+                                          TableScan [TS_0] (rows=80000000 width=860)
+                                            default@customer,customer,Tbl:COMPLETE,Col:NONE,Output:["c_customer_sk","c_current_addr_sk"]
+                                  <-Map 7 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_41]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_5] (rows=40000000 width=1014)
+                                        Output:["_col0","_col1","_col2"]
+                                        Filter Operator [FIL_76] (rows=40000000 width=1014)
+                                          predicate:ca_address_sk is not null
+                                          TableScan [TS_3] (rows=40000000 width=1014)
+                                            default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county","ca_zip"]
+


[16/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, 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/86b18772
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/86b18772
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/86b18772

Branch: refs/heads/hive-14535
Commit: 86b18772d277488b04146c9618a1da544389ae47
Parents: 824b9c8
Author: Pengcheng Xiong <px...@hortonworks.com>
Authored: Tue May 30 17:00:59 2017 -0700
Committer: Pengcheng Xiong <px...@hortonworks.com>
Committed: Tue May 30 17:00:59 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   2 +-
 .../test/queries/clientpositive/perf/query1.q   |   7 +-
 .../test/queries/clientpositive/perf/query10.q  |  61 ++
 .../test/queries/clientpositive/perf/query12.q  |  46 +-
 .../test/queries/clientpositive/perf/query13.q  | 106 +--
 .../test/queries/clientpositive/perf/query14.q  |   5 +-
 .../test/queries/clientpositive/perf/query15.q  |  23 +-
 .../test/queries/clientpositive/perf/query16.q  |   9 +-
 .../test/queries/clientpositive/perf/query17.q  |  48 +-
 .../test/queries/clientpositive/perf/query18.q  |  37 +-
 .../test/queries/clientpositive/perf/query19.q  |  28 +-
 .../test/queries/clientpositive/perf/query2.q   |  62 ++
 .../test/queries/clientpositive/perf/query20.q  |  32 +-
 .../test/queries/clientpositive/perf/query21.q  |   3 +
 .../test/queries/clientpositive/perf/query22.q  |   3 +
 .../test/queries/clientpositive/perf/query23.q  |   6 +-
 .../test/queries/clientpositive/perf/query24.q  | 100 +--
 .../test/queries/clientpositive/perf/query25.q  |  51 +-
 .../test/queries/clientpositive/perf/query26.q  |  24 +-
 .../test/queries/clientpositive/perf/query27.q  |   3 +
 .../test/queries/clientpositive/perf/query28.q  |   5 +-
 .../test/queries/clientpositive/perf/query29.q  |  50 +-
 .../test/queries/clientpositive/perf/query3.q   |  24 +-
 .../test/queries/clientpositive/perf/query30.q  |   7 +-
 .../test/queries/clientpositive/perf/query31.q  |  54 +-
 .../test/queries/clientpositive/perf/query32.q  |  47 +-
 .../test/queries/clientpositive/perf/query33.q  |   6 +-
 .../test/queries/clientpositive/perf/query34.q  |  33 +-
 .../test/queries/clientpositive/perf/query35.q  |  59 ++
 .../test/queries/clientpositive/perf/query36.q  |   9 +-
 .../test/queries/clientpositive/perf/query37.q  |   3 +
 .../test/queries/clientpositive/perf/query38.q  |   3 +
 .../test/queries/clientpositive/perf/query39.q  |  56 +-
 .../test/queries/clientpositive/perf/query40.q  |   3 +
 .../test/queries/clientpositive/perf/query42.q  |  25 +-
 .../test/queries/clientpositive/perf/query43.q  |  22 +-
 .../test/queries/clientpositive/perf/query44.q  |  37 +
 .../test/queries/clientpositive/perf/query45.q  |  22 +
 .../test/queries/clientpositive/perf/query46.q  |  38 +-
 .../test/queries/clientpositive/perf/query48.q  |  69 +-
 .../test/queries/clientpositive/perf/query49.q  | 129 +++
 .../test/queries/clientpositive/perf/query5.q   |   3 +
 .../test/queries/clientpositive/perf/query50.q  |  25 +-
 .../test/queries/clientpositive/perf/query51.q  |  14 +-
 .../test/queries/clientpositive/perf/query52.q  |  25 +-
 .../test/queries/clientpositive/perf/query53.q  |  30 +
 .../test/queries/clientpositive/perf/query54.q  |  59 +-
 .../test/queries/clientpositive/perf/query55.q  |  17 +-
 .../test/queries/clientpositive/perf/query56.q  |   7 +-
 .../test/queries/clientpositive/perf/query58.q  |  86 +-
 .../test/queries/clientpositive/perf/query59.q  |  46 +
 .../test/queries/clientpositive/perf/query6.q   |   7 +-
 .../test/queries/clientpositive/perf/query60.q  |   5 +-
 .../test/queries/clientpositive/perf/query61.q  |  46 +
 .../test/queries/clientpositive/perf/query63.q  |  31 +
 .../test/queries/clientpositive/perf/query64.q  | 121 ++-
 .../test/queries/clientpositive/perf/query65.q  |  65 +-
 .../test/queries/clientpositive/perf/query66.q  | 259 +++---
 .../test/queries/clientpositive/perf/query67.q  |   3 +-
 .../test/queries/clientpositive/perf/query68.q  |  45 +-
 .../test/queries/clientpositive/perf/query69.q  |   7 +-
 .../test/queries/clientpositive/perf/query7.q   |  24 +-
 .../test/queries/clientpositive/perf/query70.q  |  19 +-
 .../test/queries/clientpositive/perf/query71.q  |  42 +-
 .../test/queries/clientpositive/perf/query72.q  |   3 +
 .../test/queries/clientpositive/perf/query73.q  |  30 +-
 .../test/queries/clientpositive/perf/query75.q  |  73 +-
 .../test/queries/clientpositive/perf/query76.q  |  27 +-
 .../test/queries/clientpositive/perf/query77.q  | 109 +++
 .../test/queries/clientpositive/perf/query78.q  |  60 ++
 .../test/queries/clientpositive/perf/query79.q  |  26 +-
 .../test/queries/clientpositive/perf/query8.q   |   3 +
 .../test/queries/clientpositive/perf/query80.q  |   3 +
 .../test/queries/clientpositive/perf/query81.q  |   7 +-
 .../test/queries/clientpositive/perf/query82.q  |   3 +
 .../test/queries/clientpositive/perf/query83.q  |   6 +-
 .../test/queries/clientpositive/perf/query84.q  |  24 +-
 .../test/queries/clientpositive/perf/query85.q  |  87 +-
 .../test/queries/clientpositive/perf/query86.q  |   9 +-
 .../test/queries/clientpositive/perf/query87.q  |   3 +
 .../test/queries/clientpositive/perf/query88.q  |  54 +-
 .../test/queries/clientpositive/perf/query89.q  |   9 +-
 .../test/queries/clientpositive/perf/query9.q   |  19 +-
 .../test/queries/clientpositive/perf/query90.q  |  24 +-
 .../test/queries/clientpositive/perf/query91.q  |  33 +-
 .../test/queries/clientpositive/perf/query92.q  |  33 +-
 .../test/queries/clientpositive/perf/query93.q  |  21 +-
 .../test/queries/clientpositive/perf/query94.q  |  32 +-
 .../test/queries/clientpositive/perf/query95.q  |  35 +-
 .../test/queries/clientpositive/perf/query96.q  |  19 +-
 .../test/queries/clientpositive/perf/query97.q  |  28 +-
 .../test/queries/clientpositive/perf/query98.q  |  39 +-
 .../test/queries/clientpositive/perf/query99.q  |  37 +
 .../results/clientpositive/perf/query1.q.out    |   6 +-
 .../results/clientpositive/perf/query10.q.out   | 296 +++++++
 .../results/clientpositive/perf/query12.q.out   | 178 ++--
 .../results/clientpositive/perf/query13.q.out   | 204 +++--
 .../results/clientpositive/perf/query15.q.out   |  38 +-
 .../results/clientpositive/perf/query16.q.out   | 302 +++----
 .../results/clientpositive/perf/query17.q.out   |  88 +-
 .../results/clientpositive/perf/query18.q.out   |  70 +-
 .../results/clientpositive/perf/query19.q.out   |  48 +-
 .../results/clientpositive/perf/query2.q.out    | 259 ++++++
 .../results/clientpositive/perf/query20.q.out   |  74 +-
 .../results/clientpositive/perf/query23.q.out   |   8 +-
 .../results/clientpositive/perf/query24.q.out   | 190 ++--
 .../results/clientpositive/perf/query25.q.out   | 100 ++-
 .../results/clientpositive/perf/query26.q.out   |  40 +-
 .../results/clientpositive/perf/query28.q.out   |   6 +-
 .../results/clientpositive/perf/query29.q.out   |  98 ++-
 .../results/clientpositive/perf/query3.q.out    |  40 +-
 .../results/clientpositive/perf/query30.q.out   |   6 +-
 .../results/clientpositive/perf/query31.q.out   | 585 +++++++------
 .../results/clientpositive/perf/query32.q.out   | 248 +++---
 .../results/clientpositive/perf/query33.q.out   |   6 +-
 .../results/clientpositive/perf/query34.q.out   |  66 +-
 .../results/clientpositive/perf/query35.q.out   | 292 +++++++
 .../results/clientpositive/perf/query36.q.out   |  12 +-
 .../results/clientpositive/perf/query39.q.out   | 108 ++-
 .../results/clientpositive/perf/query42.q.out   | 126 ++-
 .../results/clientpositive/perf/query43.q.out   |  36 +-
 .../results/clientpositive/perf/query44.q.out   | 279 ++++++
 .../results/clientpositive/perf/query45.q.out   | 180 ++++
 .../results/clientpositive/perf/query46.q.out   |  72 +-
 .../results/clientpositive/perf/query48.q.out   | 130 ++-
 .../results/clientpositive/perf/query49.q.out   | 504 +++++++++++
 .../results/clientpositive/perf/query50.q.out   |  44 +-
 .../results/clientpositive/perf/query51.q.out   |  26 +-
 .../results/clientpositive/perf/query52.q.out   |  42 +-
 .../results/clientpositive/perf/query53.q.out   | 141 +++
 .../results/clientpositive/perf/query54.q.out   | 521 +++++++----
 .../results/clientpositive/perf/query55.q.out   |  26 +-
 .../results/clientpositive/perf/query56.q.out   |   6 +-
 .../results/clientpositive/perf/query58.q.out   | 674 ++++++++------
 .../results/clientpositive/perf/query59.q.out   | 238 +++++
 .../results/clientpositive/perf/query6.q.out    |   6 +-
 .../results/clientpositive/perf/query60.q.out   |   6 +-
 .../results/clientpositive/perf/query61.q.out   | 300 +++++++
 .../results/clientpositive/perf/query62.q.out   | 164 ++++
 .../results/clientpositive/perf/query63.q.out   | 143 +++
 .../results/clientpositive/perf/query64.q.out   | 244 +++++-
 .../results/clientpositive/perf/query65.q.out   | 124 ++-
 .../results/clientpositive/perf/query66.q.out   | 512 +++++------
 .../results/clientpositive/perf/query68.q.out   |  86 +-
 .../results/clientpositive/perf/query69.q.out   |   6 +-
 .../results/clientpositive/perf/query7.q.out    |  40 +-
 .../results/clientpositive/perf/query70.q.out   |  34 +-
 .../results/clientpositive/perf/query71.q.out   |  76 +-
 .../results/clientpositive/perf/query73.q.out   |  60 +-
 .../results/clientpositive/perf/query75.q.out   | 874 +++++++++++--------
 .../results/clientpositive/perf/query76.q.out   |  46 +-
 .../results/clientpositive/perf/query77.q.out   | 509 +++++++++++
 .../results/clientpositive/perf/query78.q.out   | 302 +++++++
 .../results/clientpositive/perf/query79.q.out   |  68 +-
 .../results/clientpositive/perf/query81.q.out   |   6 +-
 .../results/clientpositive/perf/query83.q.out   |   6 +-
 .../results/clientpositive/perf/query84.q.out   |  66 +-
 .../results/clientpositive/perf/query85.q.out   | 168 +++-
 .../results/clientpositive/perf/query86.q.out   |  12 +-
 .../results/clientpositive/perf/query88.q.out   |  96 +-
 .../results/clientpositive/perf/query89.q.out   |  12 +-
 .../results/clientpositive/perf/query9.q.out    |  26 +-
 .../results/clientpositive/perf/query90.q.out   |  42 +-
 .../results/clientpositive/perf/query91.q.out   |  62 +-
 .../results/clientpositive/perf/query92.q.out   | 229 +++--
 .../results/clientpositive/perf/query93.q.out   |  34 +-
 .../results/clientpositive/perf/query94.q.out   | 333 ++++---
 .../results/clientpositive/perf/query95.q.out   | 341 +++++---
 .../results/clientpositive/perf/query96.q.out   | 144 +--
 .../results/clientpositive/perf/query97.q.out   |  52 +-
 .../results/clientpositive/perf/query98.q.out   |  86 +-
 .../results/clientpositive/perf/query99.q.out   | 164 ++++
 172 files changed, 12021 insertions(+), 3070 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index 393df65..da52cd5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -134,7 +134,7 @@ KW_TIMESTAMPTZ: 'TIMESTAMPTZ';
 KW_TIME: 'TIME';
 KW_ZONE: 'ZONE';
 KW_INTERVAL: 'INTERVAL';
-KW_DECIMAL: 'DECIMAL' | 'DEC';
+KW_DECIMAL: 'DECIMAL' | 'DEC' | 'NUMERIC';
 KW_STRING: 'STRING';
 KW_CHAR: 'CHAR';
 KW_VARCHAR: 'VARCHAR';

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query1.q b/ql/src/test/queries/clientpositive/perf/query1.q
index d40f66a..a8d7072 100644
--- a/ql/src/test/queries/clientpositive/perf/query1.q
+++ b/ql/src/test/queries/clientpositive/perf/query1.q
@@ -1,6 +1,7 @@
 set hive.mapred.mode=nonstrict;
-
-explain with customer_total_return as
+-- start query 1 in stream 0 using template query1.tpl and seed 2031708268
+explain
+with customer_total_return as
 (select sr_customer_sk as ctr_customer_sk
 ,sr_store_sk as ctr_store_sk
 ,sum(SR_FEE) as ctr_total_return
@@ -22,3 +23,5 @@ and s_state = 'NM'
 and ctr1.ctr_customer_sk = c_customer_sk
 order by c_customer_id
 limit 100;
+
+-- end query 1 in stream 0 using template query1.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query10.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query10.q b/ql/src/test/queries/clientpositive/perf/query10.q
new file mode 100644
index 0000000..d3b1be7
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query10.q
@@ -0,0 +1,61 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query10.tpl and seed 797269820
+explain
+select  
+  cd_gender,
+  cd_marital_status,
+  cd_education_status,
+  count(*) cnt1,
+  cd_purchase_estimate,
+  count(*) cnt2,
+  cd_credit_rating,
+  count(*) cnt3,
+  cd_dep_count,
+  count(*) cnt4,
+  cd_dep_employed_count,
+  count(*) cnt5,
+  cd_dep_college_count,
+  count(*) cnt6
+ from
+  customer c,customer_address ca,customer_demographics
+ where
+  c.c_current_addr_sk = ca.ca_address_sk and
+  ca_county in ('Walker County','Richland County','Gaines County','Douglas County','Dona Ana County') and
+  cd_demo_sk = c.c_current_cdemo_sk and 
+  exists (select *
+          from store_sales,date_dim
+          where c.c_customer_sk = ss_customer_sk and
+                ss_sold_date_sk = d_date_sk and
+                d_year = 2002 and
+                d_moy between 4 and 4+3) and
+   (exists (select *
+            from web_sales,date_dim
+            where c.c_customer_sk = ws_bill_customer_sk and
+                  ws_sold_date_sk = d_date_sk and
+                  d_year = 2002 and
+                  d_moy between 4 ANd 4+3) or 
+    exists (select * 
+            from catalog_sales,date_dim
+            where c.c_customer_sk = cs_ship_customer_sk and
+                  cs_sold_date_sk = d_date_sk and
+                  d_year = 2002 and
+                  d_moy between 4 and 4+3))
+ group by cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ order by cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+limit 100;
+
+-- end query 1 in stream 0 using template query10.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query12.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query12.q b/ql/src/test/queries/clientpositive/perf/query12.q
index 7ef6cb8..59b50ac 100644
--- a/ql/src/test/queries/clientpositive/perf/query12.q
+++ b/ql/src/test/queries/clientpositive/perf/query12.q
@@ -1,13 +1,35 @@
-set hive.cbo.enable=false;
-explain 
-select 
-i_item_desc ,i_category ,i_class ,i_current_price ,i_item_id ,sum(ws_ext_sales_price) as itemrevenue ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over (partition by i_class) as revenueratio 
-from web_sales ,item ,date_dim 
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query12.tpl and seed 345591136
+explain
+select  i_item_desc 
+      ,i_category 
+      ,i_class 
+      ,i_current_price
+      ,sum(ws_ext_sales_price) as itemrevenue 
+      ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over
+          (partition by i_class) as revenueratio
+from	
+	web_sales
+    	,item 
+    	,date_dim
 where 
-web_sales.ws_item_sk = item.i_item_sk 
-and item.i_category in ('Jewelry', 'Sports', 'Books') 
-and web_sales.ws_sold_date_sk = date_dim.d_date_sk 
-and date_dim.d_date between cast('2001-01-12' as date)
-                                and (cast('2001-01-12' as date) + 30 days)
-group by i_item_id ,i_item_desc ,i_category ,i_class ,i_current_price order by i_category ,i_class ,i_item_id ,i_item_desc ,revenueratio limit 100;
-set hive.cbo.enable=true;
+	ws_item_sk = i_item_sk 
+  	and i_category in ('Jewelry', 'Sports', 'Books')
+  	and ws_sold_date_sk = d_date_sk
+	and d_date between cast('2001-01-12' as date) 
+				and (cast('2001-01-12' as date) + 30 days)
+group by 
+	i_item_id
+        ,i_item_desc 
+        ,i_category
+        ,i_class
+        ,i_current_price
+order by 
+	i_category
+        ,i_class
+        ,i_item_id
+        ,i_item_desc
+        ,revenueratio
+limit 100;
+
+-- end query 1 in stream 0 using template query12.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query13.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query13.q b/ql/src/test/queries/clientpositive/perf/query13.q
index 6f43d47..dca19b0 100644
--- a/ql/src/test/queries/clientpositive/perf/query13.q
+++ b/ql/src/test/queries/clientpositive/perf/query13.q
@@ -1,54 +1,54 @@
 set hive.mapred.mode=nonstrict;
-EXPLAIN SELECT Avg(ss_quantity) , 
-       Avg(ss_ext_sales_price) , 
-       Avg(ss_ext_wholesale_cost) , 
-       Sum(ss_ext_wholesale_cost) 
-FROM   store_sales , 
-       store , 
-       customer_demographics , 
-       household_demographics , 
-       customer_address , 
-       date_dim 
-WHERE  store.s_store_sk = store_sales.ss_store_sk 
-AND    store_sales.ss_sold_date_sk = date_dim.d_date_sk 
-AND    date_dim.d_year = 2001 
-AND   (( 
-                     store_sales.ss_hdemo_sk=household_demographics.hd_demo_sk 
-              AND    customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk 
-              AND    customer_demographics.cd_marital_status = 'M' 
-              AND    customer_demographics.cd_education_status = '4 yr Degree' 
-              AND    store_sales.ss_sales_price BETWEEN 100.00 AND    150.00 
-              AND    household_demographics.hd_dep_count = 3 ) 
-       OR     ( 
-                     store_sales.ss_hdemo_sk=household_demographics.hd_demo_sk 
-              AND    customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk 
-              AND    customer_demographics.cd_marital_status = 'D' 
-              AND    customer_demographics.cd_education_status = 'Primary' 
-              AND    store_sales.ss_sales_price BETWEEN 50.00 AND    100.00 
-              AND    household_demographics.hd_dep_count = 1 ) 
-       OR     ( 
-                     store_sales.ss_hdemo_sk=household_demographics.hd_demo_sk 
-              AND    customer_demographics.cd_demo_sk = ss_cdemo_sk 
-              AND    customer_demographics.cd_marital_status = 'U' 
-              AND    customer_demographics.cd_education_status = 'Advanced Degree' 
-              AND    store_sales.ss_sales_price BETWEEN 150.00 AND    200.00 
-              AND    household_demographics.hd_dep_count = 1 )) 
-AND   (( 
-                     store_sales.ss_addr_sk = customer_address.ca_address_sk 
-              AND    customer_address.ca_country = 'United States' 
-              AND    customer_address.ca_state IN ('KY', 
-                                                   'GA', 
-                                                   'NM') 
-              AND    store_sales.ss_net_profit BETWEEN 100 AND    200 ) 
-       OR     ( 
-                     store_sales.ss_addr_sk = customer_address.ca_address_sk 
-              AND    customer_address.ca_country = 'United States' 
-              AND    customer_address.ca_state IN ('MT', 
-                                                   'OR', 
-                                                   'IN') 
-              AND    store_sales.ss_net_profit BETWEEN 150 AND    300 ) 
-       OR     ( 
-                     store_sales.ss_addr_sk = customer_address.ca_address_sk 
-              AND    customer_address.ca_country = 'United States' 
-              AND    customer_address.ca_state IN ('WI', 'MO', 'WV') 
-              AND    store_sales.ss_net_profit BETWEEN 50 AND    250 )) ;
\ No newline at end of file
+-- start query 1 in stream 0 using template query13.tpl and seed 622697896
+explain
+select avg(ss_quantity)
+       ,avg(ss_ext_sales_price)
+       ,avg(ss_ext_wholesale_cost)
+       ,sum(ss_ext_wholesale_cost)
+ from store_sales
+     ,store
+     ,customer_demographics
+     ,household_demographics
+     ,customer_address
+     ,date_dim
+ where s_store_sk = ss_store_sk
+ and  ss_sold_date_sk = d_date_sk and d_year = 2001
+ and((ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'M'
+  and cd_education_status = '4 yr Degree'
+  and ss_sales_price between 100.00 and 150.00
+  and hd_dep_count = 3   
+     )or
+     (ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'D'
+  and cd_education_status = 'Primary'
+  and ss_sales_price between 50.00 and 100.00   
+  and hd_dep_count = 1
+     ) or 
+     (ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'U'
+  and cd_education_status = 'Advanced Degree'
+  and ss_sales_price between 150.00 and 200.00 
+  and hd_dep_count = 1  
+     ))
+ and((ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('KY', 'GA', 'NM')
+  and ss_net_profit between 100 and 200  
+     ) or
+     (ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('MT', 'OR', 'IN')
+  and ss_net_profit between 150 and 300  
+     ) or
+     (ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('WI', 'MO', 'WV')
+  and ss_net_profit between 50 and 250  
+     ))
+;
+
+-- end query 1 in stream 0 using template query13.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query14.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query14.q b/ql/src/test/queries/clientpositive/perf/query14.q
index e075f4f..c12ecb5 100644
--- a/ql/src/test/queries/clientpositive/perf/query14.q
+++ b/ql/src/test/queries/clientpositive/perf/query14.q
@@ -1,6 +1,5 @@
--- SORT_BEFORE_DIFF
-set hive.strict.checks.cartesian.product=false;
-
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query14.tpl and seed 1819994127
 explain
 with  cross_items as
  (select i_item_sk ss_item_sk

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query15.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query15.q b/ql/src/test/queries/clientpositive/perf/query15.q
index 9f75410..9e1711a 100644
--- a/ql/src/test/queries/clientpositive/perf/query15.q
+++ b/ql/src/test/queries/clientpositive/perf/query15.q
@@ -1 +1,22 @@
-explain select ca_zip ,sum(cs_sales_price) from catalog_sales ,customer ,customer_address ,date_dim where catalog_sales.cs_bill_customer_sk = customer.c_customer_sk and customer.c_current_addr_sk = customer_address.ca_address_sk and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') or customer_address.ca_state in ('CA','WA','GA') or catalog_sales.cs_sales_price > 500) and catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and date_dim.d_qoy = 2 and date_dim.d_year = 2000 group by ca_zip order by ca_zip limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query15.tpl and seed 1819994127
+explain
+select  ca_zip
+       ,sum(cs_sales_price)
+ from catalog_sales
+     ,customer
+     ,customer_address
+     ,date_dim
+ where cs_bill_customer_sk = c_customer_sk
+ 	and c_current_addr_sk = ca_address_sk 
+ 	and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475',
+                                   '85392', '85460', '80348', '81792')
+ 	      or ca_state in ('CA','WA','GA')
+ 	      or cs_sales_price > 500)
+ 	and cs_sold_date_sk = d_date_sk
+ 	and d_qoy = 2 and d_year = 2000
+ group by ca_zip
+ order by ca_zip
+ limit 100;
+
+-- end query 1 in stream 0 using template query15.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query16.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query16.q b/ql/src/test/queries/clientpositive/perf/query16.q
index 0243bf3..05625f7 100644
--- a/ql/src/test/queries/clientpositive/perf/query16.q
+++ b/ql/src/test/queries/clientpositive/perf/query16.q
@@ -1,6 +1,7 @@
 set hive.mapred.mode=nonstrict;
-
-explain select  
+-- start query 1 in stream 0 using template query16.tpl and seed 171719422
+explain
+select  
    count(distinct cs_order_number) as `order count`
   ,sum(cs_ext_ship_cost) as `total shipping cost`
   ,sum(cs_net_profit) as `total net profit`
@@ -26,5 +27,7 @@ and exists (select *
 and not exists(select *
                from catalog_returns cr1
                where cs1.cs_order_number = cr1.cr_order_number)
-order by `order count`
+order by count(distinct cs_order_number)
 limit 100;
+
+-- end query 1 in stream 0 using template query16.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query17.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query17.q b/ql/src/test/queries/clientpositive/perf/query17.q
index bb8a33b..0cd4201 100644
--- a/ql/src/test/queries/clientpositive/perf/query17.q
+++ b/ql/src/test/queries/clientpositive/perf/query17.q
@@ -1 +1,47 @@
-explain select i_item_id ,i_item_desc ,s_state ,count(ss_quantity) as store_sales_quantitycount ,avg(ss_quantity) as store_sales_quantityave ,stddev_samp(ss_quantity) as store_sales_quantitystdev ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov ,count(sr_return_quantity) as_store_returns_quantitycount ,avg(sr_return_quantity) as_store_returns_quantityave ,stddev_samp(sr_return_quantity) as_store_returns_quantitystdev ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitystdev ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov from store_sales ,store_returns ,catalog_sales ,date_dim d1 ,date_dim d2 ,date_dim d3 ,store ,item where d1.d_quarter_name = '2000Q1' and d1.d_date_sk = store_sales.ss_sold_date_sk and item.i_item_sk = store_sales.ss_item_sk 
 and store.s_store_sk = store_sales.ss_store_sk and store_sales.ss_customer_sk = store_returns.sr_customer_sk and store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number and store_returns.sr_returned_date_sk = d2.d_date_sk and d2.d_quarter_name in ('2000Q1','2000Q2','2000Q3') and store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk and store_returns.sr_item_sk = catalog_sales.cs_item_sk and catalog_sales.cs_sold_date_sk = d3.d_date_sk and d3.d_quarter_name in ('2000Q1','2000Q2','2000Q3') group by i_item_id ,i_item_desc ,s_state order by i_item_id ,i_item_desc ,s_state limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query17.tpl and seed 1819994127
+explain
+select  i_item_id
+       ,i_item_desc
+       ,s_state
+       ,count(ss_quantity) as store_sales_quantitycount
+       ,avg(ss_quantity) as store_sales_quantityave
+       ,stddev_samp(ss_quantity) as store_sales_quantitystdev
+       ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov
+       ,count(sr_return_quantity) as_store_returns_quantitycount
+       ,avg(sr_return_quantity) as_store_returns_quantityave
+       ,stddev_samp(sr_return_quantity) as_store_returns_quantitystdev
+       ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov
+       ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave
+       ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitystdev
+       ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov
+ from store_sales
+     ,store_returns
+     ,catalog_sales
+     ,date_dim d1
+     ,date_dim d2
+     ,date_dim d3
+     ,store
+     ,item
+ where d1.d_quarter_name = '2000Q1'
+   and d1.d_date_sk = ss_sold_date_sk
+   and i_item_sk = ss_item_sk
+   and s_store_sk = ss_store_sk
+   and ss_customer_sk = sr_customer_sk
+   and ss_item_sk = sr_item_sk
+   and ss_ticket_number = sr_ticket_number
+   and sr_returned_date_sk = d2.d_date_sk
+   and d2.d_quarter_name in ('2000Q1','2000Q2','2000Q3')
+   and sr_customer_sk = cs_bill_customer_sk
+   and sr_item_sk = cs_item_sk
+   and cs_sold_date_sk = d3.d_date_sk
+   and d3.d_quarter_name in ('2000Q1','2000Q2','2000Q3')
+ group by i_item_id
+         ,i_item_desc
+         ,s_state
+ order by i_item_id
+         ,i_item_desc
+         ,s_state
+limit 100;
+
+-- end query 1 in stream 0 using template query17.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query18.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query18.q b/ql/src/test/queries/clientpositive/perf/query18.q
index e6061f5..bf1ff59 100644
--- a/ql/src/test/queries/clientpositive/perf/query18.q
+++ b/ql/src/test/queries/clientpositive/perf/query18.q
@@ -1 +1,36 @@
-explain select i_item_id, ca_country, ca_state, ca_county, avg( cast(cs_quantity as decimal(12,2))) agg1, avg( cast(cs_list_price as decimal(12,2))) agg2, avg( cast(cs_coupon_amt as decimal(12,2))) agg3, avg( cast(cs_sales_price as decimal(12,2))) agg4, avg( cast(cs_net_profit as decimal(12,2))) agg5, avg( cast(c_birth_year as decimal(12,2))) agg6, avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 from catalog_sales, date_dim, customer_demographics cd1, item, customer, customer_address, customer_demographics cd2 where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and catalog_sales.cs_item_sk = item.i_item_sk and catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk and cd1.cd_gender = 'M' and cd1.cd_education_status = 'College' and customer.c_current_cdemo_sk = cd2.cd_demo_sk and customer.c_current_addr_sk = customer_address.ca_address_sk and c_birth_month in (9,5,12,4,1,10) and d_year = 2001 and ca_state in ('ND','WI','
 AL' ,'NC','OK','MS','TN') group by i_item_id, ca_country, ca_state, ca_county with rollup order by ca_country, ca_state, ca_county, i_item_id limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query18.tpl and seed 1978355063
+explain
+select  i_item_id,
+        ca_country,
+        ca_state, 
+        ca_county,
+        avg( cast(cs_quantity as numeric(12,2))) agg1,
+        avg( cast(cs_list_price as numeric(12,2))) agg2,
+        avg( cast(cs_coupon_amt as numeric(12,2))) agg3,
+        avg( cast(cs_sales_price as numeric(12,2))) agg4,
+        avg( cast(cs_net_profit as numeric(12,2))) agg5,
+        avg( cast(c_birth_year as numeric(12,2))) agg6,
+        avg( cast(cd1.cd_dep_count as numeric(12,2))) agg7
+ from catalog_sales, customer_demographics cd1, 
+      customer_demographics cd2, customer, customer_address, date_dim, item
+ where cs_sold_date_sk = d_date_sk and
+       cs_item_sk = i_item_sk and
+       cs_bill_cdemo_sk = cd1.cd_demo_sk and
+       cs_bill_customer_sk = c_customer_sk and
+       cd1.cd_gender = 'M' and 
+       cd1.cd_education_status = 'College' and
+       c_current_cdemo_sk = cd2.cd_demo_sk and
+       c_current_addr_sk = ca_address_sk and
+       c_birth_month in (9,5,12,4,1,10) and
+       d_year = 2001 and
+       ca_state in ('ND','WI','AL'
+                   ,'NC','OK','MS','TN')
+ group by rollup (i_item_id, ca_country, ca_state, ca_county)
+ order by ca_country,
+        ca_state, 
+        ca_county,
+	i_item_id
+ limit 100;
+
+-- end query 1 in stream 0 using template query18.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query19.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query19.q b/ql/src/test/queries/clientpositive/perf/query19.q
index cfb6218..5768e4b 100644
--- a/ql/src/test/queries/clientpositive/perf/query19.q
+++ b/ql/src/test/queries/clientpositive/perf/query19.q
@@ -1 +1,27 @@
-explain select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, sum(ss_ext_sales_price) ext_price from date_dim, store_sales, item,customer,customer_address,store where date_dim.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and i_manager_id=7 and d_moy=11 and d_year=1999 and store_sales.ss_customer_sk = customer.c_customer_sk and customer.c_current_addr_sk = customer_address.ca_address_sk and substr(ca_zip,1,5) <> substr(s_zip,1,5) and store_sales.ss_store_sk = store.s_store_sk group by i_brand ,i_brand_id ,i_manufact_id ,i_manufact order by ext_price desc ,i_brand ,i_brand_id ,i_manufact_id ,i_manufact limit 100 ;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query19.tpl and seed 1930872976
+explain
+select  i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact,
+ 	sum(ss_ext_sales_price) ext_price
+ from date_dim, store_sales, item,customer,customer_address,store
+ where d_date_sk = ss_sold_date_sk
+   and ss_item_sk = i_item_sk
+   and i_manager_id=7
+   and d_moy=11
+   and d_year=1999
+   and ss_customer_sk = c_customer_sk 
+   and c_current_addr_sk = ca_address_sk
+   and substr(ca_zip,1,5) <> substr(s_zip,1,5) 
+   and ss_store_sk = s_store_sk 
+ group by i_brand
+      ,i_brand_id
+      ,i_manufact_id
+      ,i_manufact
+ order by ext_price desc
+         ,i_brand
+         ,i_brand_id
+         ,i_manufact_id
+         ,i_manufact
+limit 100 ;
+
+-- end query 1 in stream 0 using template query19.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query2.q b/ql/src/test/queries/clientpositive/perf/query2.q
new file mode 100644
index 0000000..26a52ef
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query2.q
@@ -0,0 +1,62 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query2.tpl and seed 1819994127
+explain
+with wscs as
+ (select sold_date_sk
+        ,sales_price
+  from (select ws_sold_date_sk sold_date_sk
+              ,ws_ext_sales_price sales_price
+        from web_sales) x
+        union all
+       (select cs_sold_date_sk sold_date_sk
+              ,cs_ext_sales_price sales_price
+        from catalog_sales)),
+ wswscs as 
+ (select d_week_seq,
+        sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales
+ from wscs
+     ,date_dim
+ where d_date_sk = sold_date_sk
+ group by d_week_seq)
+ select d_week_seq1
+       ,round(sun_sales1/sun_sales2,2)
+       ,round(mon_sales1/mon_sales2,2)
+       ,round(tue_sales1/tue_sales2,2)
+       ,round(wed_sales1/wed_sales2,2)
+       ,round(thu_sales1/thu_sales2,2)
+       ,round(fri_sales1/fri_sales2,2)
+       ,round(sat_sales1/sat_sales2,2)
+ from
+ (select wswscs.d_week_seq d_week_seq1
+        ,sun_sales sun_sales1
+        ,mon_sales mon_sales1
+        ,tue_sales tue_sales1
+        ,wed_sales wed_sales1
+        ,thu_sales thu_sales1
+        ,fri_sales fri_sales1
+        ,sat_sales sat_sales1
+  from wswscs,date_dim 
+  where date_dim.d_week_seq = wswscs.d_week_seq and
+        d_year = 2001) y,
+ (select wswscs.d_week_seq d_week_seq2
+        ,sun_sales sun_sales2
+        ,mon_sales mon_sales2
+        ,tue_sales tue_sales2
+        ,wed_sales wed_sales2
+        ,thu_sales thu_sales2
+        ,fri_sales fri_sales2
+        ,sat_sales sat_sales2
+  from wswscs
+      ,date_dim 
+  where date_dim.d_week_seq = wswscs.d_week_seq and
+        d_year = 2001+1) z
+ where d_week_seq1=d_week_seq2-53
+ order by d_week_seq1;
+
+-- end query 1 in stream 0 using template query2.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query20.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query20.q b/ql/src/test/queries/clientpositive/perf/query20.q
index d4ea10d..c5f8848 100644
--- a/ql/src/test/queries/clientpositive/perf/query20.q
+++ b/ql/src/test/queries/clientpositive/perf/query20.q
@@ -1 +1,31 @@
-explain select i_item_desc ,i_category ,i_class ,i_current_price ,i_item_id ,sum(cs_ext_sales_price) as itemrevenue ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over (partition by i_class) as revenueratio from catalog_sales ,item ,date_dim where catalog_sales.cs_item_sk = item.i_item_sk and i_category in ('Jewelry', 'Sports', 'Books') and catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and d_date between '2001-01-12' and '2001-02-11' group by i_item_id ,i_item_desc ,i_category ,i_class ,i_current_price order by i_category ,i_class ,i_item_id ,i_item_desc ,revenueratio limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query20.tpl and seed 345591136
+explain
+select  i_item_desc 
+       ,i_category 
+       ,i_class 
+       ,i_current_price
+       ,sum(cs_ext_sales_price) as itemrevenue 
+       ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over
+           (partition by i_class) as revenueratio
+ from	catalog_sales
+     ,item 
+     ,date_dim
+ where cs_item_sk = i_item_sk 
+   and i_category in ('Jewelry', 'Sports', 'Books')
+   and cs_sold_date_sk = d_date_sk
+ and d_date between cast('2001-01-12' as date) 
+ 				and (cast('2001-01-12' as date) + 30 days)
+ group by i_item_id
+         ,i_item_desc 
+         ,i_category
+         ,i_class
+         ,i_current_price
+ order by i_category
+         ,i_class
+         ,i_item_id
+         ,i_item_desc
+         ,revenueratio
+limit 100;
+
+-- end query 1 in stream 0 using template query20.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query21.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query21.q b/ql/src/test/queries/clientpositive/perf/query21.q
index 4c68df9..34b458b 100644
--- a/ql/src/test/queries/clientpositive/perf/query21.q
+++ b/ql/src/test/queries/clientpositive/perf/query21.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query21.tpl and seed 1819994127
 explain
 select  *
  from(select w_warehouse_name
@@ -27,3 +29,4 @@ select  *
          ,i_item_id
  limit 100;
 
+-- end query 1 in stream 0 using template query21.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query22.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query22.q b/ql/src/test/queries/clientpositive/perf/query22.q
index adb509d..7049173 100644
--- a/ql/src/test/queries/clientpositive/perf/query22.q
+++ b/ql/src/test/queries/clientpositive/perf/query22.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query22.tpl and seed 1819994127
 explain
 select  i_product_name
              ,i_brand
@@ -19,3 +21,4 @@ select  i_product_name
 order by qoh, i_product_name, i_brand, i_class, i_category
 limit 100;
 
+-- end query 1 in stream 0 using template query22.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query23.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query23.q b/ql/src/test/queries/clientpositive/perf/query23.q
index 4939416..1e02655 100644
--- a/ql/src/test/queries/clientpositive/perf/query23.q
+++ b/ql/src/test/queries/clientpositive/perf/query23.q
@@ -1,7 +1,7 @@
--- SORT_BEFORE_DIFF
 set hive.mapred.mode=nonstrict;
-
-explain with frequent_ss_items as 
+-- start query 1 in stream 0 using template query23.tpl and seed 2031708268
+explain
+with frequent_ss_items as 
  (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
   from store_sales
       ,date_dim 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query24.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query24.q b/ql/src/test/queries/clientpositive/perf/query24.q
index b9b747f..007d7ee 100644
--- a/ql/src/test/queries/clientpositive/perf/query24.q
+++ b/ql/src/test/queries/clientpositive/perf/query24.q
@@ -1,51 +1,51 @@
 set hive.mapred.mode=nonstrict;
-
-explain with ssales as
-        (select c_last_name
-              ,c_first_name
-              ,s_store_name
-              ,ca_state
-              ,s_state
-              ,i_color
-              ,i_current_price
-              ,i_manager_id
-              ,i_units
-              ,i_size
-              ,sum(ss_sales_price) netpaid
-        from store_sales
-            ,store_returns
-            ,store
-            ,item
-            ,customer
-            ,customer_address
-        where ss_ticket_number = sr_ticket_number
-          and ss_item_sk = sr_item_sk
-          and ss_customer_sk = c_customer_sk
-          and ss_item_sk = i_item_sk
-          and ss_store_sk = s_store_sk
-          and c_birth_country = upper(ca_country)
-          and s_zip = ca_zip
-        and s_market_id=7
-        group by c_last_name
-                ,c_first_name
-                ,s_store_name
-                ,ca_state
-                ,s_state
-                ,i_color
-                ,i_current_price
-                ,i_manager_id
-                ,i_units
-                ,i_size)
-        select c_last_name
-              ,c_first_name
-              ,s_store_name
-              ,sum(netpaid) paid
-        from ssales
-        where i_color = 'orchid'
-        group by c_last_name
-                ,c_first_name
-                ,s_store_name
-        having sum(netpaid) > (select 0.05*avg(netpaid)
-                                         from ssales)
-        ;
-
+-- start query 1 in stream 0 using template query24.tpl and seed 1220860970
+explain
+with ssales as
+(select c_last_name
+      ,c_first_name
+      ,s_store_name
+      ,ca_state
+      ,s_state
+      ,i_color
+      ,i_current_price
+      ,i_manager_id
+      ,i_units
+      ,i_size
+      ,sum(ss_sales_price) netpaid
+from store_sales
+    ,store_returns
+    ,store
+    ,item
+    ,customer
+    ,customer_address
+where ss_ticket_number = sr_ticket_number
+  and ss_item_sk = sr_item_sk
+  and ss_customer_sk = c_customer_sk
+  and ss_item_sk = i_item_sk
+  and ss_store_sk = s_store_sk
+  and c_birth_country = upper(ca_country)
+  and s_zip = ca_zip
+and s_market_id=7
+group by c_last_name
+        ,c_first_name
+        ,s_store_name
+        ,ca_state
+        ,s_state
+        ,i_color
+        ,i_current_price
+        ,i_manager_id
+        ,i_units
+        ,i_size)
+select c_last_name
+      ,c_first_name
+      ,s_store_name
+      ,sum(netpaid) paid
+from ssales
+where i_color = 'orchid'
+group by c_last_name
+        ,c_first_name
+        ,s_store_name
+having sum(netpaid) > (select 0.05*avg(netpaid)
+                                 from ssales)
+;

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query25.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query25.q b/ql/src/test/queries/clientpositive/perf/query25.q
index 80185ab..358cdc5 100644
--- a/ql/src/test/queries/clientpositive/perf/query25.q
+++ b/ql/src/test/queries/clientpositive/perf/query25.q
@@ -1 +1,50 @@
-explain select i_item_id ,i_item_desc ,s_store_id ,s_store_name ,sum(ss_net_profit) as store_sales_profit ,sum(sr_net_loss) as store_returns_loss ,sum(cs_net_profit) as catalog_sales_profit from store_sales ,store_returns ,catalog_sales ,date_dim d1 ,date_dim d2 ,date_dim d3 ,store ,item where d1.d_moy = 4 and d1.d_year = 1998 and d1.d_date_sk = ss_sold_date_sk and i_item_sk = ss_item_sk and s_store_sk = ss_store_sk and ss_customer_sk = sr_customer_sk and ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number and sr_returned_date_sk = d2.d_date_sk and d2.d_moy between 4 and 10 and d2.d_year = 1998 and sr_customer_sk = cs_bill_customer_sk and sr_item_sk = cs_item_sk and cs_sold_date_sk = d3.d_date_sk and d3.d_moy between 4 and 10 and d3.d_year = 1998 group by i_item_id ,i_item_desc ,s_store_id ,s_store_name order by i_item_id ,i_item_desc ,s_store_id ,s_store_name limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query25.tpl and seed 1819994127
+explain
+select  
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ ,sum(ss_net_profit) as store_sales_profit
+ ,sum(sr_net_loss) as store_returns_loss
+ ,sum(cs_net_profit) as catalog_sales_profit
+ from
+ store_sales
+ ,store_returns
+ ,catalog_sales
+ ,date_dim d1
+ ,date_dim d2
+ ,date_dim d3
+ ,store
+ ,item
+ where
+ d1.d_moy = 4
+ and d1.d_year = 2000
+ and d1.d_date_sk = ss_sold_date_sk
+ and i_item_sk = ss_item_sk
+ and s_store_sk = ss_store_sk
+ and ss_customer_sk = sr_customer_sk
+ and ss_item_sk = sr_item_sk
+ and ss_ticket_number = sr_ticket_number
+ and sr_returned_date_sk = d2.d_date_sk
+ and d2.d_moy               between 4 and  10
+ and d2.d_year              = 2000
+ and sr_customer_sk = cs_bill_customer_sk
+ and sr_item_sk = cs_item_sk
+ and cs_sold_date_sk = d3.d_date_sk
+ and d3.d_moy               between 4 and  10 
+ and d3.d_year              = 2000
+ group by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ order by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ limit 100;
+
+-- end query 1 in stream 0 using template query25.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query26.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query26.q b/ql/src/test/queries/clientpositive/perf/query26.q
index 0e81120..b35d98c 100644
--- a/ql/src/test/queries/clientpositive/perf/query26.q
+++ b/ql/src/test/queries/clientpositive/perf/query26.q
@@ -1 +1,23 @@
-explain select i_item_id, avg(cs_quantity) agg1, avg(cs_list_price) agg2, avg(cs_coupon_amt) agg3, avg(cs_sales_price) agg4 from catalog_sales, customer_demographics, date_dim, item, promotion where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and catalog_sales.cs_item_sk = item.i_item_sk and catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk and catalog_sales.cs_promo_sk = promotion.p_promo_sk and cd_gender = 'F' and cd_marital_status = 'W' and cd_education_status = 'Primary' and (p_channel_email = 'N' or p_channel_event = 'N') and d_year = 1998 group by i_item_id order by i_item_id limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query26.tpl and seed 1930872976
+explain
+select  i_item_id, 
+        avg(cs_quantity) agg1,
+        avg(cs_list_price) agg2,
+        avg(cs_coupon_amt) agg3,
+        avg(cs_sales_price) agg4 
+ from catalog_sales, customer_demographics, date_dim, item, promotion
+ where cs_sold_date_sk = d_date_sk and
+       cs_item_sk = i_item_sk and
+       cs_bill_cdemo_sk = cd_demo_sk and
+       cs_promo_sk = p_promo_sk and
+       cd_gender = 'F' and 
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       (p_channel_email = 'N' or p_channel_event = 'N') and
+       d_year = 1998 
+ group by i_item_id
+ order by i_item_id
+ limit 100;
+
+-- end query 1 in stream 0 using template query26.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query27.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query27.q b/ql/src/test/queries/clientpositive/perf/query27.q
index 0cbb3ae..ec09e1d 100644
--- a/ql/src/test/queries/clientpositive/perf/query27.q
+++ b/ql/src/test/queries/clientpositive/perf/query27.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query27.tpl and seed 2017787633
 explain
 select  i_item_id,
         s_state, grouping(s_state) g_state,
@@ -20,3 +22,4 @@ select  i_item_id,
          ,s_state
  limit 100;
 
+-- end query 1 in stream 0 using template query27.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query28.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query28.q b/ql/src/test/queries/clientpositive/perf/query28.q
index 2a5f092..b0cdfac 100644
--- a/ql/src/test/queries/clientpositive/perf/query28.q
+++ b/ql/src/test/queries/clientpositive/perf/query28.q
@@ -1,5 +1,7 @@
 set hive.mapred.mode=nonstrict;
-explain select  *
+-- start query 1 in stream 0 using template query28.tpl and seed 444293455
+explain
+select  *
 from (select avg(ss_list_price) B1_LP
             ,count(ss_list_price) B1_CNT
             ,count(distinct ss_list_price) B1_CNTD
@@ -50,3 +52,4 @@ from (select avg(ss_list_price) B1_LP
           or ss_wholesale_cost between 42 and 42+20)) B6
 limit 100;
 
+-- end query 1 in stream 0 using template query28.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query29.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query29.q b/ql/src/test/queries/clientpositive/perf/query29.q
index 1634b1f..8bf4d51 100644
--- a/ql/src/test/queries/clientpositive/perf/query29.q
+++ b/ql/src/test/queries/clientpositive/perf/query29.q
@@ -1 +1,49 @@
-explain select i_item_id ,i_item_desc ,s_store_id ,s_store_name ,sum(ss_quantity) as store_sales_quantity ,sum(sr_return_quantity) as store_returns_quantity ,sum(cs_quantity) as catalog_sales_quantity from store_sales ,store_returns ,catalog_sales ,date_dim d1 ,date_dim d2 ,date_dim d3 ,store ,item where d1.d_moy = 2 and d1.d_year = 2000 and d1.d_date_sk = ss_sold_date_sk and i_item_sk = ss_item_sk and s_store_sk = ss_store_sk and ss_customer_sk = sr_customer_sk and ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number and sr_returned_date_sk = d2.d_date_sk and d2.d_moy between 2 and 2 + 3 and d2.d_year = 2000 and sr_customer_sk = cs_bill_customer_sk and sr_item_sk = cs_item_sk and cs_sold_date_sk = d3.d_date_sk and d3.d_year in (2000,2000+1,2000+2) group by i_item_id ,i_item_desc ,s_store_id ,s_store_name order by i_item_id ,i_item_desc ,s_store_id ,s_store_name limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query29.tpl and seed 2031708268
+explain
+select   
+     i_item_id
+    ,i_item_desc
+    ,s_store_id
+    ,s_store_name
+    ,sum(ss_quantity)        as store_sales_quantity
+    ,sum(sr_return_quantity) as store_returns_quantity
+    ,sum(cs_quantity)        as catalog_sales_quantity
+ from
+    store_sales
+   ,store_returns
+   ,catalog_sales
+   ,date_dim             d1
+   ,date_dim             d2
+   ,date_dim             d3
+   ,store
+   ,item
+ where
+     d1.d_moy               = 4 
+ and d1.d_year              = 1999
+ and d1.d_date_sk           = ss_sold_date_sk
+ and i_item_sk              = ss_item_sk
+ and s_store_sk             = ss_store_sk
+ and ss_customer_sk         = sr_customer_sk
+ and ss_item_sk             = sr_item_sk
+ and ss_ticket_number       = sr_ticket_number
+ and sr_returned_date_sk    = d2.d_date_sk
+ and d2.d_moy               between 4 and  4 + 3 
+ and d2.d_year              = 1999
+ and sr_customer_sk         = cs_bill_customer_sk
+ and sr_item_sk             = cs_item_sk
+ and cs_sold_date_sk        = d3.d_date_sk     
+ and d3.d_year              in (1999,1999+1,1999+2)
+ group by
+    i_item_id
+   ,i_item_desc
+   ,s_store_id
+   ,s_store_name
+ order by
+    i_item_id 
+   ,i_item_desc
+   ,s_store_id
+   ,s_store_name
+ limit 100;
+
+-- end query 1 in stream 0 using template query29.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query3.q b/ql/src/test/queries/clientpositive/perf/query3.q
index 47dfac5..a70a62f 100644
--- a/ql/src/test/queries/clientpositive/perf/query3.q
+++ b/ql/src/test/queries/clientpositive/perf/query3.q
@@ -1 +1,23 @@
-explain select dt.d_year ,item.i_brand_id brand_id ,item.i_brand brand ,sum(ss_ext_sales_price) sum_agg from date_dim dt ,store_sales ,item where dt.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and item.i_manufact_id = 436 and dt.d_moy=12 group by dt.d_year ,item.i_brand ,item.i_brand_id order by dt.d_year ,sum_agg desc ,brand_id limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query3.tpl and seed 2031708268
+explain
+select  dt.d_year 
+       ,item.i_brand_id brand_id 
+       ,item.i_brand brand
+       ,sum(ss_ext_sales_price) sum_agg
+ from  date_dim dt 
+      ,store_sales
+      ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+   and store_sales.ss_item_sk = item.i_item_sk
+   and item.i_manufact_id = 436
+   and dt.d_moy=12
+ group by dt.d_year
+      ,item.i_brand
+      ,item.i_brand_id
+ order by dt.d_year
+         ,sum_agg desc
+         ,brand_id
+ limit 100;
+
+-- end query 1 in stream 0 using template query3.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query30.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query30.q b/ql/src/test/queries/clientpositive/perf/query30.q
index 25c9e07..47f0d93 100644
--- a/ql/src/test/queries/clientpositive/perf/query30.q
+++ b/ql/src/test/queries/clientpositive/perf/query30.q
@@ -1,6 +1,7 @@
 set hive.mapred.mode=nonstrict;
-
-explain with customer_total_return as
+-- start query 1 in stream 0 using template query30.tpl and seed 1819994127
+explain
+with customer_total_return as
  (select wr_returning_customer_sk as ctr_customer_sk
         ,ca_state as ctr_state, 
  	sum(wr_return_amt) as ctr_total_return
@@ -28,3 +29,5 @@ explain with customer_total_return as
                   ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address
                   ,c_last_review_date,ctr_total_return
 limit 100;
+
+-- end query 1 in stream 0 using template query30.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query31.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query31.q b/ql/src/test/queries/clientpositive/perf/query31.q
index d448dc3..42c3ca6 100644
--- a/ql/src/test/queries/clientpositive/perf/query31.q
+++ b/ql/src/test/queries/clientpositive/perf/query31.q
@@ -1,2 +1,54 @@
 set hive.mapred.mode=nonstrict;
-explain with ss as (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales from store_sales,date_dim,customer_address where ss_sold_date_sk = d_date_sk and ss_addr_sk=ca_address_sk group by ca_county,d_qoy, d_year), ws as (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales from web_sales,date_dim,customer_address where ws_sold_date_sk = d_date_sk and ws_bill_addr_sk=ca_address_sk group by ca_county,d_qoy, d_year) select ss1.ca_county ,ss1.d_year ,ws2.web_sales/ws1.web_sales web_q1_q2_increase ,ss2.store_sales/ss1.store_sales store_q1_q2_increase ,ws3.web_sales/ws2.web_sales web_q2_q3_increase ,ss3.store_sales/ss2.store_sales store_q2_q3_increase from ss ss1 ,ss ss2 ,ss ss3 ,ws ws1 ,ws ws2 ,ws ws3 where ss1.d_qoy = 1 and ss1.d_year = 1998 and ss1.ca_county = ss2.ca_county and ss2.d_qoy = 2 and ss2.d_year = 1998 and ss2.ca_county = ss3.ca_county and ss3.d_qoy = 3 and ss3.d_year = 1998 and ss1.ca_county = ws1.ca_county and ws1.d_qoy = 1 and ws1.d_year = 
 1998 and ws1.ca_county = ws2.ca_county and ws2.d_qoy = 2 and ws2.d_year = 1998 and ws1.ca_county = ws3.ca_county and ws3.d_qoy = 3 and ws3.d_year =1998 and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end order by web_q1_q2_increase;
+-- start query 1 in stream 0 using template query31.tpl and seed 1819994127
+explain
+with ss as
+ (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales
+ from store_sales,date_dim,customer_address
+ where ss_sold_date_sk = d_date_sk
+  and ss_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year),
+ ws as
+ (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales
+ from web_sales,date_dim,customer_address
+ where ws_sold_date_sk = d_date_sk
+  and ws_bill_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year)
+ select /* tt */
+        ss1.ca_county
+       ,ss1.d_year
+       ,ws2.web_sales/ws1.web_sales web_q1_q2_increase
+       ,ss2.store_sales/ss1.store_sales store_q1_q2_increase
+       ,ws3.web_sales/ws2.web_sales web_q2_q3_increase
+       ,ss3.store_sales/ss2.store_sales store_q2_q3_increase
+ from
+        ss ss1
+       ,ss ss2
+       ,ss ss3
+       ,ws ws1
+       ,ws ws2
+       ,ws ws3
+ where
+    ss1.d_qoy = 1
+    and ss1.d_year = 2000
+    and ss1.ca_county = ss2.ca_county
+    and ss2.d_qoy = 2
+    and ss2.d_year = 2000
+ and ss2.ca_county = ss3.ca_county
+    and ss3.d_qoy = 3
+    and ss3.d_year = 2000
+    and ss1.ca_county = ws1.ca_county
+    and ws1.d_qoy = 1
+    and ws1.d_year = 2000
+    and ws1.ca_county = ws2.ca_county
+    and ws2.d_qoy = 2
+    and ws2.d_year = 2000
+    and ws1.ca_county = ws3.ca_county
+    and ws3.d_qoy = 3
+    and ws3.d_year =2000
+    and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end 
+       > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end
+    and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end
+       > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end
+ order by ss1.d_year;
+
+-- end query 1 in stream 0 using template query31.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query32.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query32.q b/ql/src/test/queries/clientpositive/perf/query32.q
index ceb2ddb..ed43b4d 100644
--- a/ql/src/test/queries/clientpositive/perf/query32.q
+++ b/ql/src/test/queries/clientpositive/perf/query32.q
@@ -1,19 +1,30 @@
-explain SELECT sum(cs1.cs_ext_discount_amt) as excess_discount_amount
-FROM (SELECT cs.cs_item_sk as cs_item_sk,
-                             cs.cs_ext_discount_amt as cs_ext_discount_amt
-             FROM catalog_sales cs
-             JOIN date_dim d ON (d.d_date_sk = cs.cs_sold_date_sk)
-             WHERE d.d_date between '2000-01-27' and '2000-04-27') cs1
-JOIN item i ON (i.i_item_sk = cs1.cs_item_sk)
-JOIN (SELECT cs2.cs_item_sk as cs_item_sk,
-                          1.3 * avg(cs_ext_discount_amt) as avg_cs_ext_discount_amt
-           FROM (SELECT cs.cs_item_sk as cs_item_sk,
-                                        cs.cs_ext_discount_amt as cs_ext_discount_amt
-                        FROM catalog_sales cs
-                        JOIN date_dim d ON (d.d_date_sk = cs.cs_sold_date_sk)
-                        WHERE d.d_date between '2000-01-27' and '2000-04-27') cs2
-                        GROUP BY cs2.cs_item_sk) tmp1
-ON (i.i_item_sk = tmp1.cs_item_sk)
-WHERE i.i_manufact_id = 436 and
-               cs1.cs_ext_discount_amt > tmp1.avg_cs_ext_discount_amt;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query32.tpl and seed 2031708268
+explain
+select  sum(cs_ext_discount_amt)  as `excess discount amount` 
+from 
+   catalog_sales 
+   ,item 
+   ,date_dim
+where
+i_manufact_id = 269
+and i_item_sk = cs_item_sk 
+and d_date between '1998-03-18' and 
+        (cast('1998-03-18' as date) + 90 days)
+and d_date_sk = cs_sold_date_sk 
+and cs_ext_discount_amt  
+     > ( 
+         select 
+            1.3 * avg(cs_ext_discount_amt) 
+         from 
+            catalog_sales 
+           ,date_dim
+         where 
+              cs_item_sk = i_item_sk 
+          and d_date between '1998-03-18' and
+                             (cast('1998-03-18' as date) + 90 days)
+          and d_date_sk = cs_sold_date_sk 
+      ) 
+limit 100;
 
+-- end query 1 in stream 0 using template query32.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query33.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query33.q b/ql/src/test/queries/clientpositive/perf/query33.q
index 06628bd..1dfa9be 100644
--- a/ql/src/test/queries/clientpositive/perf/query33.q
+++ b/ql/src/test/queries/clientpositive/perf/query33.q
@@ -1,5 +1,7 @@
+set hive.mapred.mode=nonstrict;
 -- start query 1 in stream 0 using template query33.tpl and seed 1930872976
-explain with ss as (
+explain
+with ss as (
  select
           i_manufact_id,sum(ss_ext_sales_price) total_sales
  from
@@ -71,3 +73,5 @@ where i_category in ('Books'))
  group by i_manufact_id
  order by total_sales
 limit 100;
+
+-- end query 1 in stream 0 using template query33.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query34.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query34.q b/ql/src/test/queries/clientpositive/perf/query34.q
index 5c3efe1..427eed6 100644
--- a/ql/src/test/queries/clientpositive/perf/query34.q
+++ b/ql/src/test/queries/clientpositive/perf/query34.q
@@ -1,2 +1,33 @@
 set hive.mapred.mode=nonstrict;
-explain select c_last_name ,c_first_name ,c_salutation ,c_preferred_cust_flag ,ss_ticket_number ,cnt from (select ss_ticket_number ,ss_customer_sk ,count(*) cnt from store_sales,date_dim,store,household_demographics where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) and (household_demographics.hd_buy_potential = '1001-5000' or household_demographics.hd_buy_potential = '5001-10000') and household_demographics.hd_vehicle_count > 0 and (case when household_demographics.hd_vehicle_count > 0 then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end) > 1.2 and date_dim.d_year in (1998,1998+1,1998+2) and store.s_county in ('Kittitas County','Adams County','Richland County','Furnas County', 'Orange County','Appanoose County','Franklin Parish','Tehama County') group by 
 ss_ticket_number,ss_customer_sk) dn,customer where dn.ss_customer_sk = customer.c_customer_sk and cnt between 15 and 20 order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc;
+-- start query 1 in stream 0 using template query34.tpl and seed 1971067816
+explain
+select c_last_name
+       ,c_first_name
+       ,c_salutation
+       ,c_preferred_cust_flag
+       ,ss_ticket_number
+       ,cnt from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,count(*) cnt
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28)
+    and (household_demographics.hd_buy_potential = '>10000' or
+         household_demographics.hd_buy_potential = 'unknown')
+    and household_demographics.hd_vehicle_count > 0
+    and (case when household_demographics.hd_vehicle_count > 0 
+	then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count 
+	else null 
+	end)  > 1.2
+    and date_dim.d_year in (2000,2000+1,2000+2)
+    and store.s_county in ('Mobile County','Maverick County','Huron County','Kittitas County',
+                           'Fairfield County','Jackson County','Barrow County','Pennington County')
+    group by ss_ticket_number,ss_customer_sk) dn,customer
+    where ss_customer_sk = c_customer_sk
+      and cnt between 15 and 20
+    order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc;
+
+-- end query 1 in stream 0 using template query34.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query35.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query35.q b/ql/src/test/queries/clientpositive/perf/query35.q
new file mode 100644
index 0000000..19951ac
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query35.q
@@ -0,0 +1,59 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query35.tpl and seed 1930872976
+explain
+select   
+  ca_state,
+  cd_gender,
+  cd_marital_status,
+  count(*) cnt1,
+  avg(cd_dep_count),
+  max(cd_dep_count),
+  sum(cd_dep_count),
+  cd_dep_employed_count,
+  count(*) cnt2,
+  avg(cd_dep_employed_count),
+  max(cd_dep_employed_count),
+  sum(cd_dep_employed_count),
+  cd_dep_college_count,
+  count(*) cnt3,
+  avg(cd_dep_college_count),
+  max(cd_dep_college_count),
+  sum(cd_dep_college_count)
+ from
+  customer c,customer_address ca,customer_demographics
+ where
+  c.c_current_addr_sk = ca.ca_address_sk and
+  cd_demo_sk = c.c_current_cdemo_sk and 
+  exists (select *
+          from store_sales,date_dim
+          where c.c_customer_sk = ss_customer_sk and
+                ss_sold_date_sk = d_date_sk and
+                d_year = 1999 and
+                d_qoy < 4) and
+   (exists (select *
+            from web_sales,date_dim
+            where c.c_customer_sk = ws_bill_customer_sk and
+                  ws_sold_date_sk = d_date_sk and
+                  d_year = 1999 and
+                  d_qoy < 4) or 
+    exists (select * 
+            from catalog_sales,date_dim
+            where c.c_customer_sk = cs_ship_customer_sk and
+                  cs_sold_date_sk = d_date_sk and
+                  d_year = 1999 and
+                  d_qoy < 4))
+ group by ca_state,
+          cd_gender,
+          cd_marital_status,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ order by ca_state,
+          cd_gender,
+          cd_marital_status,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ limit 100;
+
+-- end query 1 in stream 0 using template query35.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query36.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query36.q b/ql/src/test/queries/clientpositive/perf/query36.q
index 6c3a945..789f932 100644
--- a/ql/src/test/queries/clientpositive/perf/query36.q
+++ b/ql/src/test/queries/clientpositive/perf/query36.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query36.tpl and seed 1544728811
 explain
 select  
     sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin
@@ -5,9 +7,9 @@ select
    ,i_class
    ,grouping(i_category)+grouping(i_class) as lochierarchy
    ,rank() over (
-     partition by grouping(i_category)+grouping(i_class),
-     case when grouping(i_class) = 0 then i_category end 
-     order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent
+ 	partition by grouping(i_category)+grouping(i_class),
+ 	case when grouping(i_class) = 0 then i_category end 
+ 	order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent
  from
     store_sales
    ,date_dim       d1
@@ -27,3 +29,4 @@ select
   ,rank_within_parent
   limit 100;
 
+-- end query 1 in stream 0 using template query36.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query37.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query37.q b/ql/src/test/queries/clientpositive/perf/query37.q
index c71ec56..811eab0 100644
--- a/ql/src/test/queries/clientpositive/perf/query37.q
+++ b/ql/src/test/queries/clientpositive/perf/query37.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query37.tpl and seed 301843662
 explain
 select  i_item_id
        ,i_item_desc
@@ -14,3 +16,4 @@ select  i_item_id
  order by i_item_id
  limit 100;
 
+-- end query 1 in stream 0 using template query37.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query38.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query38.q b/ql/src/test/queries/clientpositive/perf/query38.q
index 2e8517e..8eade8a 100644
--- a/ql/src/test/queries/clientpositive/perf/query38.q
+++ b/ql/src/test/queries/clientpositive/perf/query38.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query38.tpl and seed 1819994127
 explain
 select  count(*) from (
     select distinct c_last_name, c_first_name, d_date
@@ -20,3 +22,4 @@ select  count(*) from (
 ) hot_cust
 limit 100;
 
+-- end query 1 in stream 0 using template query38.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query39.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query39.q b/ql/src/test/queries/clientpositive/perf/query39.q
index c9f8f2b..d3c806d 100644
--- a/ql/src/test/queries/clientpositive/perf/query39.q
+++ b/ql/src/test/queries/clientpositive/perf/query39.q
@@ -1,2 +1,56 @@
 set hive.mapred.mode=nonstrict;
-explain with inv as (select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stdev,mean, case mean when 0 then null else stdev/mean end cov from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean from inventory ,item ,warehouse ,date_dim where inv_item_sk = i_item_sk and inv_warehouse_sk = w_warehouse_sk and inv_date_sk = d_date_sk and d_year =1999 group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo where case mean when 0 then 0 else stdev/mean end > 1) select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov from inv inv1,inv inv2 where inv1.i_item_sk = inv2.i_item_sk and inv1.w_warehouse_sk = inv2.w_warehouse_sk and inv1.d_moy=3 and inv2.d_moy=3+1 order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov ,inv2.d_moy,inv2.mean, inv2.cov ; with inv as (select w_warehouse_name,w_warehouse_sk,i_item_sk,
 d_moy ,stdev,mean, case mean when 0 then null else stdev/mean end cov from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean from inventory ,item ,warehouse ,date_dim where inv_item_sk = i_item_sk and inv_warehouse_sk = w_warehouse_sk and inv_date_sk = d_date_sk and d_year =1999 group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo where case mean when 0 then 0 else stdev/mean end > 1) select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov from inv inv1,inv inv2 where inv1.i_item_sk = inv2.i_item_sk and inv1.w_warehouse_sk = inv2.w_warehouse_sk and inv1.d_moy=3 and inv2.d_moy=3+1 and inv1.cov > 1.5 order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov ,inv2.d_moy,inv2.mean, inv2.cov ;
+-- start query 1 in stream 0 using template query39.tpl and seed 1327317894
+explain
+with inv as
+(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+       ,stdev,mean, case mean when 0 then null else stdev/mean end cov
+ from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+            ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean
+      from inventory
+          ,item
+          ,warehouse
+          ,date_dim
+      where inv_item_sk = i_item_sk
+        and inv_warehouse_sk = w_warehouse_sk
+        and inv_date_sk = d_date_sk
+        and d_year =1999
+      group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo
+ where case mean when 0 then 0 else stdev/mean end > 1)
+select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov
+        ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov
+from inv inv1,inv inv2
+where inv1.i_item_sk = inv2.i_item_sk
+  and inv1.w_warehouse_sk =  inv2.w_warehouse_sk
+  and inv1.d_moy=4
+  and inv2.d_moy=4+1
+order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov
+        ,inv2.d_moy,inv2.mean, inv2.cov
+;
+with inv as
+(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+       ,stdev,mean, case mean when 0 then null else stdev/mean end cov
+ from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+            ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean
+      from inventory
+          ,item
+          ,warehouse
+          ,date_dim
+      where inv_item_sk = i_item_sk
+        and inv_warehouse_sk = w_warehouse_sk
+        and inv_date_sk = d_date_sk
+        and d_year =1999
+      group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo
+ where case mean when 0 then 0 else stdev/mean end > 1)
+select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov
+        ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov
+from inv inv1,inv inv2
+where inv1.i_item_sk = inv2.i_item_sk
+  and inv1.w_warehouse_sk =  inv2.w_warehouse_sk
+  and inv1.d_moy=4
+  and inv2.d_moy=4+1
+  and inv1.cov > 1.5
+order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov
+        ,inv2.d_moy,inv2.mean, inv2.cov
+;
+
+-- end query 1 in stream 0 using template query39.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query40.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query40.q b/ql/src/test/queries/clientpositive/perf/query40.q
index 155fa08..61f5ad3 100644
--- a/ql/src/test/queries/clientpositive/perf/query40.q
+++ b/ql/src/test/queries/clientpositive/perf/query40.q
@@ -1,3 +1,5 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query40.tpl and seed 1819994127
 explain
 select  
    w_state
@@ -25,3 +27,4 @@ select
  order by w_state,i_item_id
 limit 100;
 
+-- end query 1 in stream 0 using template query40.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query42.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query42.q b/ql/src/test/queries/clientpositive/perf/query42.q
index c4b115b..6b8abe0 100644
--- a/ql/src/test/queries/clientpositive/perf/query42.q
+++ b/ql/src/test/queries/clientpositive/perf/query42.q
@@ -1 +1,24 @@
-explain select dt.d_year ,item.i_category_id ,item.i_category ,sum(ss_ext_sales_price) as s from date_dim dt ,store_sales ,item where dt.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and item.i_manager_id = 1 and dt.d_moy=12 and dt.d_year=1998 group by dt.d_year ,item.i_category_id ,item.i_category order by s desc,dt.d_year ,item.i_category_id ,item.i_category limit 100 ;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query42.tpl and seed 1819994127
+explain
+select  dt.d_year
+ 	,item.i_category_id
+ 	,item.i_category
+ 	,sum(ss_ext_sales_price)
+ from 	date_dim dt
+ 	,store_sales
+ 	,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+ 	and store_sales.ss_item_sk = item.i_item_sk
+ 	and item.i_manager_id = 1  	
+ 	and dt.d_moy=12
+ 	and dt.d_year=1998
+ group by 	dt.d_year
+ 		,item.i_category_id
+ 		,item.i_category
+ order by       sum(ss_ext_sales_price) desc,dt.d_year
+ 		,item.i_category_id
+ 		,item.i_category
+limit 100 ;
+
+-- end query 1 in stream 0 using template query42.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query43.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query43.q b/ql/src/test/queries/clientpositive/perf/query43.q
index f1c22e8..ebdc69d 100644
--- a/ql/src/test/queries/clientpositive/perf/query43.q
+++ b/ql/src/test/queries/clientpositive/perf/query43.q
@@ -1 +1,21 @@
-explain select s_store_name, s_store_id, sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales from date_dim, store_sales, store where date_dim.d_date_sk = store_sales.ss_sold_date_sk and store.s_store_sk = store_sales.ss_store_sk and s_gmt_offset = -6 and d_year = 1998 group by s_store_name, s_store_id order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query43.tpl and seed 1819994127
+explain
+select  s_store_name, s_store_id,
+        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from date_dim, store_sales, store
+ where d_date_sk = ss_sold_date_sk and
+       s_store_sk = ss_store_sk and
+       s_gmt_offset = -6 and
+       d_year = 1998 
+ group by s_store_name, s_store_id
+ order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales
+ limit 100;
+
+-- end query 1 in stream 0 using template query43.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query44.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query44.q b/ql/src/test/queries/clientpositive/perf/query44.q
new file mode 100644
index 0000000..712bbfb
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query44.q
@@ -0,0 +1,37 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query44.tpl and seed 1819994127
+explain
+select  asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing
+from(select *
+     from (select item_sk,rank() over (order by rank_col asc) rnk
+           from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col 
+                 from store_sales ss1
+                 where ss_store_sk = 410
+                 group by ss_item_sk
+                 having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col
+                                                  from store_sales
+                                                  where ss_store_sk = 410
+                                                    and ss_hdemo_sk is null
+                                                  group by ss_store_sk))V1)V11
+     where rnk  < 11) asceding,
+    (select *
+     from (select item_sk,rank() over (order by rank_col desc) rnk
+           from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col
+                 from store_sales ss1
+                 where ss_store_sk = 410
+                 group by ss_item_sk
+                 having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col
+                                                  from store_sales
+                                                  where ss_store_sk = 410
+                                                    and ss_hdemo_sk is null
+                                                  group by ss_store_sk))V2)V21
+     where rnk  < 11) descending,
+item i1,
+item i2
+where asceding.rnk = descending.rnk 
+  and i1.i_item_sk=asceding.item_sk
+  and i2.i_item_sk=descending.item_sk
+order by asceding.rnk
+limit 100;
+
+-- end query 1 in stream 0 using template query44.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query45.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query45.q b/ql/src/test/queries/clientpositive/perf/query45.q
new file mode 100644
index 0000000..4db3fb2
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/perf/query45.q
@@ -0,0 +1,22 @@
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query45.tpl and seed 2031708268
+explain
+select  ca_zip, ca_county, sum(ws_sales_price)
+ from web_sales, customer, customer_address, date_dim, item
+ where ws_bill_customer_sk = c_customer_sk
+ 	and c_current_addr_sk = ca_address_sk 
+ 	and ws_item_sk = i_item_sk 
+ 	and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792')
+ 	      or 
+ 	      i_item_id in (select i_item_id
+                             from item
+                             where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)
+                             )
+ 	    )
+ 	and ws_sold_date_sk = d_date_sk
+ 	and d_qoy = 2 and d_year = 2000
+ group by ca_zip, ca_county
+ order by ca_zip, ca_county
+ limit 100;
+
+-- end query 1 in stream 0 using template query45.tpl

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/queries/clientpositive/perf/query46.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/perf/query46.q b/ql/src/test/queries/clientpositive/perf/query46.q
index 3e8711f..46f8be3 100644
--- a/ql/src/test/queries/clientpositive/perf/query46.q
+++ b/ql/src/test/queries/clientpositive/perf/query46.q
@@ -1 +1,37 @@
-explain select c_last_name ,c_first_name ,ca_city ,bought_city ,ss_ticket_number ,amt,profit from (select ss_ticket_number ,ss_customer_sk ,ca_city bought_city ,sum(ss_coupon_amt) amt ,sum(ss_net_profit) profit from store_sales,date_dim,store,household_demographics,customer_address where store_sales.ss_sold_date_sk = date_dim.d_date_sk and store_sales.ss_store_sk = store.s_store_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and store_sales.ss_addr_sk = customer_address.ca_address_sk and (household_demographics.hd_dep_count = 4 or household_demographics.hd_vehicle_count= 2) and date_dim.d_dow in (6,0) and date_dim.d_year in (1998,1998+1,1998+2) and store.s_city in ('Rosedale','Bethlehem','Clinton','Clifton','Springfield') group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr where dn.ss_customer_sk = customer.c_customer_sk and customer.c_current_addr_sk = current_addr.ca_address_sk and current_addr.ca_city <> bough
 t_city order by c_last_name ,c_first_name ,ca_city ,bought_city ,ss_ticket_number limit 100;
+set hive.mapred.mode=nonstrict;
+-- start query 1 in stream 0 using template query46.tpl and seed 803547492
+explain
+select  c_last_name
+       ,c_first_name
+       ,ca_city
+       ,bought_city
+       ,ss_ticket_number
+       ,amt,profit 
+ from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,ca_city bought_city
+          ,sum(ss_coupon_amt) amt
+          ,sum(ss_net_profit) profit
+    from store_sales,date_dim,store,household_demographics,customer_address 
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and store_sales.ss_addr_sk = customer_address.ca_address_sk
+    and (household_demographics.hd_dep_count = 2 or
+         household_demographics.hd_vehicle_count= 1)
+    and date_dim.d_dow in (6,0)
+    and date_dim.d_year in (1998,1998+1,1998+2) 
+    and store.s_city in ('Cedar Grove','Wildwood','Union','Salem','Highland Park') 
+    group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr
+    where ss_customer_sk = c_customer_sk
+      and customer.c_current_addr_sk = current_addr.ca_address_sk
+      and current_addr.ca_city <> bought_city
+  order by c_last_name
+          ,c_first_name
+          ,ca_city
+          ,bought_city
+          ,ss_ticket_number
+  limit 100;
+
+-- end query 1 in stream 0 using template query46.tpl


[05/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query75.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query75.q.out b/ql/src/test/results/clientpositive/perf/query75.q.out
index 55da3a2..d399567 100644
--- a/ql/src/test/results/clientpositive/perf/query75.q.out
+++ b/ql/src/test/results/clientpositive/perf/query75.q.out
@@ -1,377 +1,533 @@
-PREHOOK: query: explain WITH all_sales AS ( SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,SUM(sales_cnt) AS sales_cnt ,SUM(sales_amt) AS sales_amt FROM (SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk JOIN date_dim ON d_date_sk=cs_sold_date_sk LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number AND cs_item_sk=cr_item_sk) WHERE i_category='Sports' UNION ALL SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt FROM store_sales JOIN item ON i_item_sk=ss_item_sk JOIN date_dim ON d_date_sk=ss_sold_date_sk LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number AND ss_item_sk=sr_item_sk) WHERE i_category='Sports' UNION AL
 L SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt FROM web_sales JOIN item ON i_item_sk=ws_item_sk JOIN date_dim ON d_date_sk=ws_sold_date_sk LEFT JOIN web_returns ON (ws_order_number=wr_order_number AND ws_item_sk=wr_item_sk) WHERE i_category='Sports') sales_detail GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) SELECT prev_yr.d_year AS prev_year ,curr_yr.d_year AS year ,curr_yr.i_brand_id ,curr_yr.i_class_id ,curr_yr.i_category_id ,curr_yr.i_manufact_id ,prev_yr.sales_cnt AS prev_yr_cnt ,curr_yr.sales_cnt AS curr_yr_cnt ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff FROM all_sales curr_yr, all_sales prev_yr WHERE curr_yr.i_brand_id=prev_yr.i_brand_id AND curr_yr.i_class_id=prev_yr.i_class_id AND curr_yr.i_category_id=prev_yr.i_category_id AND curr_yr.i_manu
 fact_id=prev_yr.i_manufact_id AND curr_yr.d_year=2002 AND prev_yr.d_year=2002-1 AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9 ORDER BY sales_cnt_diff limit 100
+PREHOOK: query: explain
+WITH all_sales AS (
+ SELECT d_year
+       ,i_brand_id
+       ,i_class_id
+       ,i_category_id
+       ,i_manufact_id
+       ,SUM(sales_cnt) AS sales_cnt
+       ,SUM(sales_amt) AS sales_amt
+ FROM (SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt
+             ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt
+       FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk
+                          JOIN date_dim ON d_date_sk=cs_sold_date_sk
+                          LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number 
+                                                    AND cs_item_sk=cr_item_sk)
+       WHERE i_category='Sports'
+       UNION
+       SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt
+             ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt
+       FROM store_sales JOIN item ON i_item_sk=ss_item_sk
+                        JOIN date_dim ON d_date_sk=ss_sold_date_sk
+                        LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number 
+                                                AND ss_item_sk=sr_item_sk)
+       WHERE i_category='Sports'
+       UNION
+       SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt
+             ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt
+       FROM web_sales JOIN item ON i_item_sk=ws_item_sk
+                      JOIN date_dim ON d_date_sk=ws_sold_date_sk
+                      LEFT JOIN web_returns ON (ws_order_number=wr_order_number 
+                                            AND ws_item_sk=wr_item_sk)
+       WHERE i_category='Sports') sales_detail
+ GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id)
+ SELECT  prev_yr.d_year AS prev_year
+                          ,curr_yr.d_year AS year
+                          ,curr_yr.i_brand_id
+                          ,curr_yr.i_class_id
+                          ,curr_yr.i_category_id
+                          ,curr_yr.i_manufact_id
+                          ,prev_yr.sales_cnt AS prev_yr_cnt
+                          ,curr_yr.sales_cnt AS curr_yr_cnt
+                          ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff
+                          ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff
+ FROM all_sales curr_yr, all_sales prev_yr
+ WHERE curr_yr.i_brand_id=prev_yr.i_brand_id
+   AND curr_yr.i_class_id=prev_yr.i_class_id
+   AND curr_yr.i_category_id=prev_yr.i_category_id
+   AND curr_yr.i_manufact_id=prev_yr.i_manufact_id
+   AND curr_yr.d_year=2002
+   AND prev_yr.d_year=2002-1
+   AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9
+ ORDER BY sales_cnt_diff
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain WITH all_sales AS ( SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,SUM(sales_cnt) AS sales_cnt ,SUM(sales_amt) AS sales_amt FROM (SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk JOIN date_dim ON d_date_sk=cs_sold_date_sk LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number AND cs_item_sk=cr_item_sk) WHERE i_category='Sports' UNION ALL SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt FROM store_sales JOIN item ON i_item_sk=ss_item_sk JOIN date_dim ON d_date_sk=ss_sold_date_sk LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number AND ss_item_sk=sr_item_sk) WHERE i_category='Sports' UNION A
 LL SELECT d_year ,i_brand_id ,i_class_id ,i_category_id ,i_manufact_id ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt FROM web_sales JOIN item ON i_item_sk=ws_item_sk JOIN date_dim ON d_date_sk=ws_sold_date_sk LEFT JOIN web_returns ON (ws_order_number=wr_order_number AND ws_item_sk=wr_item_sk) WHERE i_category='Sports') sales_detail GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) SELECT prev_yr.d_year AS prev_year ,curr_yr.d_year AS year ,curr_yr.i_brand_id ,curr_yr.i_class_id ,curr_yr.i_category_id ,curr_yr.i_manufact_id ,prev_yr.sales_cnt AS prev_yr_cnt ,curr_yr.sales_cnt AS curr_yr_cnt ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff FROM all_sales curr_yr, all_sales prev_yr WHERE curr_yr.i_brand_id=prev_yr.i_brand_id AND curr_yr.i_class_id=prev_yr.i_class_id AND curr_yr.i_category_id=prev_yr.i_category_id AND curr_yr.i_man
 ufact_id=prev_yr.i_manufact_id AND curr_yr.d_year=2002 AND prev_yr.d_year=2002-1 AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9 ORDER BY sales_cnt_diff limit 100
+POSTHOOK: query: explain
+WITH all_sales AS (
+ SELECT d_year
+       ,i_brand_id
+       ,i_class_id
+       ,i_category_id
+       ,i_manufact_id
+       ,SUM(sales_cnt) AS sales_cnt
+       ,SUM(sales_amt) AS sales_amt
+ FROM (SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt
+             ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt
+       FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk
+                          JOIN date_dim ON d_date_sk=cs_sold_date_sk
+                          LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number 
+                                                    AND cs_item_sk=cr_item_sk)
+       WHERE i_category='Sports'
+       UNION
+       SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt
+             ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt
+       FROM store_sales JOIN item ON i_item_sk=ss_item_sk
+                        JOIN date_dim ON d_date_sk=ss_sold_date_sk
+                        LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number 
+                                                AND ss_item_sk=sr_item_sk)
+       WHERE i_category='Sports'
+       UNION
+       SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt
+             ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt
+       FROM web_sales JOIN item ON i_item_sk=ws_item_sk
+                      JOIN date_dim ON d_date_sk=ws_sold_date_sk
+                      LEFT JOIN web_returns ON (ws_order_number=wr_order_number 
+                                            AND ws_item_sk=wr_item_sk)
+       WHERE i_category='Sports') sales_detail
+ GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id)
+ SELECT  prev_yr.d_year AS prev_year
+                          ,curr_yr.d_year AS year
+                          ,curr_yr.i_brand_id
+                          ,curr_yr.i_class_id
+                          ,curr_yr.i_category_id
+                          ,curr_yr.i_manufact_id
+                          ,prev_yr.sales_cnt AS prev_yr_cnt
+                          ,curr_yr.sales_cnt AS curr_yr_cnt
+                          ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff
+                          ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff
+ FROM all_sales curr_yr, all_sales prev_yr
+ WHERE curr_yr.i_brand_id=prev_yr.i_brand_id
+   AND curr_yr.i_class_id=prev_yr.i_class_id
+   AND curr_yr.i_category_id=prev_yr.i_category_id
+   AND curr_yr.i_manufact_id=prev_yr.i_manufact_id
+   AND curr_yr.d_year=2002
+   AND prev_yr.d_year=2002-1
+   AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9
+ ORDER BY sales_cnt_diff
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 10 <- Map 27 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 11 <- Map 28 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE), Union 12 (CONTAINS)
-Reducer 13 <- Union 12 (SIMPLE_EDGE)
-Reducer 15 <- Map 14 (SIMPLE_EDGE), Map 29 (SIMPLE_EDGE)
-Reducer 16 <- Map 27 (SIMPLE_EDGE), Reducer 15 (SIMPLE_EDGE)
-Reducer 17 <- Map 30 (SIMPLE_EDGE), Reducer 16 (SIMPLE_EDGE), Union 5 (CONTAINS)
-Reducer 18 <- Map 14 (SIMPLE_EDGE), Map 31 (SIMPLE_EDGE)
-Reducer 19 <- Map 27 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE)
-Reducer 20 <- Map 32 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE), Union 5 (CONTAINS)
-Reducer 21 <- Map 14 (SIMPLE_EDGE), Map 29 (SIMPLE_EDGE)
-Reducer 22 <- Map 27 (SIMPLE_EDGE), Reducer 21 (SIMPLE_EDGE)
-Reducer 23 <- Map 30 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE), Union 12 (CONTAINS)
-Reducer 24 <- Map 14 (SIMPLE_EDGE), Map 31 (SIMPLE_EDGE)
-Reducer 25 <- Map 27 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
-Reducer 26 <- Map 32 (SIMPLE_EDGE), Reducer 25 (SIMPLE_EDGE), Union 12 (CONTAINS)
-Reducer 3 <- Map 27 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 28 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS)
-Reducer 6 <- Union 5 (SIMPLE_EDGE)
-Reducer 7 <- Reducer 13 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
-Reducer 9 <- Map 1 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE)
+Reducer 10 <- Reducer 9 (SIMPLE_EDGE)
+Reducer 11 <- Map 1 (SIMPLE_EDGE), Map 18 (SIMPLE_EDGE)
+Reducer 12 <- Map 31 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE)
+Reducer 13 <- Map 32 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE), Union 14 (CONTAINS)
+Reducer 15 <- Union 14 (SIMPLE_EDGE), Union 16 (CONTAINS)
+Reducer 17 <- Union 16 (SIMPLE_EDGE)
+Reducer 19 <- Map 18 (SIMPLE_EDGE), Map 33 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 18 (SIMPLE_EDGE)
+Reducer 20 <- Map 31 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE)
+Reducer 21 <- Map 34 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE), Union 5 (CONTAINS)
+Reducer 22 <- Map 18 (SIMPLE_EDGE), Map 35 (SIMPLE_EDGE)
+Reducer 23 <- Map 31 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE)
+Reducer 24 <- Map 36 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE), Union 7 (CONTAINS)
+Reducer 25 <- Map 18 (SIMPLE_EDGE), Map 33 (SIMPLE_EDGE)
+Reducer 26 <- Map 31 (SIMPLE_EDGE), Reducer 25 (SIMPLE_EDGE)
+Reducer 27 <- Map 34 (SIMPLE_EDGE), Reducer 26 (SIMPLE_EDGE), Union 14 (CONTAINS)
+Reducer 28 <- Map 18 (SIMPLE_EDGE), Map 35 (SIMPLE_EDGE)
+Reducer 29 <- Map 31 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE)
+Reducer 3 <- Map 31 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 30 <- Map 36 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE), Union 16 (CONTAINS)
+Reducer 4 <- Map 32 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS)
+Reducer 6 <- Union 5 (SIMPLE_EDGE), Union 7 (CONTAINS)
+Reducer 8 <- Union 7 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 17 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 8
-      File Output Operator [FS_157]
-        Select Operator [SEL_156] (rows=100 width=108)
+      Reducer 10
+      File Output Operator [FS_179]
+        Select Operator [SEL_178] (rows=100 width=111)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
-          Limit [LIM_155] (rows=100 width=108)
+          Limit [LIM_177] (rows=100 width=111)
             Number of rows:100
-            Select Operator [SEL_154] (rows=245965926 width=108)
+            Select Operator [SEL_176] (rows=70276244 width=111)
               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
-            <-Reducer 7 [SIMPLE_EDGE]
-              SHUFFLE [RS_153]
-                Select Operator [SEL_152] (rows=245965926 width=108)
+            <-Reducer 9 [SIMPLE_EDGE]
+              SHUFFLE [RS_175]
+                Select Operator [SEL_174] (rows=70276244 width=111)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
-                  Filter Operator [FIL_151] (rows=245965926 width=108)
+                  Filter Operator [FIL_173] (rows=70276244 width=111)
                     predicate:((CAST( _col10 AS decimal(17,2)) / CAST( _col4 AS decimal(17,2))) < 0.9)
-                    Merge Join Operator [MERGEJOIN_260] (rows=737897778 width=108)
-                      Conds:RS_148._col0, _col1, _col2, _col3=RS_149._col0, _col1, _col2, _col3(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col10","_col11"]
-                    <-Reducer 13 [SIMPLE_EDGE]
-                      SHUFFLE [RS_149]
+                    Merge Join Operator [MERGEJOIN_282] (rows=210828734 width=111)
+                      Conds:RS_170._col0, _col1, _col2, _col3=RS_171._col0, _col1, _col2, _col3(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col10","_col11"]
+                    <-Reducer 17 [SIMPLE_EDGE]
+                      SHUFFLE [RS_171]
                         PartitionCols:_col0, _col1, _col2, _col3
-                        Group By Operator [GBY_146] (rows=670816148 width=108)
-                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
-                        <-Union 12 [SIMPLE_EDGE]
-                          <-Reducer 11 [CONTAINS]
-                            Reduce Output Operator [RS_145]
-                              PartitionCols:_col0, _col1, _col2, _col3
-                              Group By Operator [GBY_144] (rows=1341632296 width=108)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3
-                                Select Operator [SEL_95] (rows=383314495 width=135)
-                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                  Merge Join Operator [MERGEJOIN_253] (rows=383314495 width=135)
-                                    Conds:RS_92._col1, _col2=RS_93._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
-                                  <-Map 28 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_93]
-                                      PartitionCols:_col0, _col1
-                                      Select Operator [SEL_85] (rows=28798881 width=106)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_233] (rows=28798881 width=106)
-                                          predicate:cr_item_sk is not null
-                                          TableScan [TS_9] (rows=28798881 width=106)
-                                            default@catalog_returns,catalog_returns,Tbl:COMPLETE,Col:NONE,Output:["cr_item_sk","cr_order_number","cr_return_quantity","cr_return_amount"]
-                                  <-Reducer 10 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_92]
-                                      PartitionCols:_col1, _col2
-                                      Merge Join Operator [MERGEJOIN_252] (rows=348467716 width=135)
-                                        Conds:RS_89._col1=RS_90._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
-                                      <-Map 27 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_90]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_82] (rows=231000 width=1436)
-                                            Output:["_col0","_col1","_col2","_col3","_col5"]
-                                            Filter Operator [FIL_232] (rows=231000 width=1436)
-                                              predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                              TableScan [TS_6] (rows=462000 width=1436)
-                                                default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
-                                      <-Reducer 9 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_89]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_251] (rows=316788826 width=135)
-                                            Conds:RS_86._col0=RS_87._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                                          <-Map 14 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_87]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_79] (rows=36524 width=1119)
-                                                Output:["_col0"]
-                                                Filter Operator [FIL_231] (rows=36524 width=1119)
-                                                  predicate:((d_year = 2002) and d_date_sk is not null)
-                                                  TableScan [TS_3] (rows=73049 width=1119)
-                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
-                                          <-Map 1 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_86]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_76] (rows=287989836 width=135)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_230] (rows=287989836 width=135)
-                                                  predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
-                                                  TableScan [TS_0] (rows=287989836 width=135)
-                                                    default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_order_number","cs_quantity","cs_ext_sales_price"]
-                          <-Reducer 23 [CONTAINS]
-                            Reduce Output Operator [RS_145]
-                              PartitionCols:_col0, _col1, _col2, _col3
-                              Group By Operator [GBY_144] (rows=1341632296 width=108)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3
-                                Select Operator [SEL_117] (rows=766650239 width=88)
-                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                  Merge Join Operator [MERGEJOIN_256] (rows=766650239 width=88)
-                                    Conds:RS_114._col1, _col2=RS_115._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
-                                  <-Map 30 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_115]
-                                      PartitionCols:_col0, _col1
-                                      Select Operator [SEL_107] (rows=57591150 width=77)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_237] (rows=57591150 width=77)
-                                          predicate:sr_item_sk is not null
-                                          TableScan [TS_31] (rows=57591150 width=77)
-                                            default@store_returns,store_returns,Tbl:COMPLETE,Col:NONE,Output:["sr_item_sk","sr_ticket_number","sr_return_quantity","sr_return_amt"]
-                                  <-Reducer 22 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_114]
-                                      PartitionCols:_col1, _col2
-                                      Merge Join Operator [MERGEJOIN_255] (rows=696954748 width=88)
-                                        Conds:RS_111._col1=RS_112._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
-                                      <-Map 27 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_112]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_104] (rows=231000 width=1436)
-                                            Output:["_col0","_col1","_col2","_col3","_col5"]
-                                            Filter Operator [FIL_236] (rows=231000 width=1436)
-                                              predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                               Please refer to the previous TableScan [TS_6]
-                                      <-Reducer 21 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_111]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_254] (rows=633595212 width=88)
-                                            Conds:RS_108._col0=RS_109._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                                          <-Map 14 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_109]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_101] (rows=36524 width=1119)
-                                                Output:["_col0"]
-                                                Filter Operator [FIL_235] (rows=36524 width=1119)
-                                                  predicate:((d_year = 2002) and d_date_sk is not null)
-                                                   Please refer to the previous TableScan [TS_3]
-                                          <-Map 29 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_108]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_98] (rows=575995635 width=88)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_234] (rows=575995635 width=88)
-                                                  predicate:(ss_item_sk is not null and ss_sold_date_sk is not null)
-                                                  TableScan [TS_22] (rows=575995635 width=88)
-                                                    default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ticket_number","ss_quantity","ss_ext_sales_price"]
-                          <-Reducer 26 [CONTAINS]
-                            Reduce Output Operator [RS_145]
-                              PartitionCols:_col0, _col1, _col2, _col3
-                              Group By Operator [GBY_144] (rows=1341632296 width=108)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3
-                                Select Operator [SEL_141] (rows=191667562 width=135)
-                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                  Merge Join Operator [MERGEJOIN_259] (rows=191667562 width=135)
-                                    Conds:RS_138._col1, _col2=RS_139._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
-                                  <-Map 32 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_139]
-                                      PartitionCols:_col0, _col1
-                                      Select Operator [SEL_131] (rows=14398467 width=92)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_241] (rows=14398467 width=92)
-                                          predicate:wr_item_sk is not null
-                                          TableScan [TS_55] (rows=14398467 width=92)
-                                            default@web_returns,web_returns,Tbl:COMPLETE,Col:NONE,Output:["wr_item_sk","wr_order_number","wr_return_quantity","wr_return_amt"]
-                                  <-Reducer 25 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_138]
-                                      PartitionCols:_col1, _col2
-                                      Merge Join Operator [MERGEJOIN_258] (rows=174243235 width=135)
-                                        Conds:RS_135._col1=RS_136._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
-                                      <-Map 27 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_136]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_128] (rows=231000 width=1436)
-                                            Output:["_col0","_col1","_col2","_col3","_col5"]
-                                            Filter Operator [FIL_240] (rows=231000 width=1436)
-                                              predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                               Please refer to the previous TableScan [TS_6]
-                                      <-Reducer 24 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_135]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_257] (rows=158402938 width=135)
-                                            Conds:RS_132._col0=RS_133._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                                          <-Map 14 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_133]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_125] (rows=36524 width=1119)
-                                                Output:["_col0"]
-                                                Filter Operator [FIL_239] (rows=36524 width=1119)
-                                                  predicate:((d_year = 2002) and d_date_sk is not null)
-                                                   Please refer to the previous TableScan [TS_3]
-                                          <-Map 31 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_132]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_122] (rows=144002668 width=135)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_238] (rows=144002668 width=135)
-                                                  predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
-                                                  TableScan [TS_46] (rows=144002668 width=135)
-                                                    default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_order_number","ws_quantity","ws_ext_sales_price"]
-                    <-Reducer 6 [SIMPLE_EDGE]
-                      SHUFFLE [RS_148]
+                        Group By Operator [GBY_168] (rows=191662482 width=111)
+                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3
+                          Group By Operator [GBY_163] (rows=383324964 width=111)
+                            Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5
+                          <-Union 16 [SIMPLE_EDGE]
+                            <-Reducer 15 [CONTAINS]
+                              Reduce Output Operator [RS_162]
+                                PartitionCols:_col0, _col1, _col2, _col3
+                                Group By Operator [GBY_161] (rows=766649929 width=111)
+                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:_col0, _col1, _col2, _col3, _col4, _col5
+                                  Group By Operator [GBY_134] (rows=574982367 width=103)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5
+                                  <-Union 14 [SIMPLE_EDGE]
+                                    <-Reducer 13 [CONTAINS]
+                                      Reduce Output Operator [RS_133]
+                                        PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5
+                                        Group By Operator [GBY_132] (rows=1149964734 width=103)
+                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:_col0, _col1, _col2, _col3, _col4, _col5
+                                          Select Operator [SEL_106] (rows=383314495 width=135)
+                                            Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                            Merge Join Operator [MERGEJOIN_275] (rows=383314495 width=135)
+                                              Conds:RS_103._col1, _col2=RS_104._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
+                                            <-Map 32 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_104]
+                                                PartitionCols:_col0, _col1
+                                                Select Operator [SEL_96] (rows=28798881 width=106)
+                                                  Output:["_col0","_col1","_col2","_col3"]
+                                                  Filter Operator [FIL_255] (rows=28798881 width=106)
+                                                    predicate:cr_item_sk is not null
+                                                    TableScan [TS_9] (rows=28798881 width=106)
+                                                      default@catalog_returns,catalog_returns,Tbl:COMPLETE,Col:NONE,Output:["cr_item_sk","cr_order_number","cr_return_quantity","cr_return_amount"]
+                                            <-Reducer 12 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_103]
+                                                PartitionCols:_col1, _col2
+                                                Merge Join Operator [MERGEJOIN_274] (rows=348467716 width=135)
+                                                  Conds:RS_100._col1=RS_101._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
+                                                <-Map 31 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_101]
+                                                    PartitionCols:_col0
+                                                    Select Operator [SEL_93] (rows=231000 width=1436)
+                                                      Output:["_col0","_col1","_col2","_col3","_col5"]
+                                                      Filter Operator [FIL_254] (rows=231000 width=1436)
+                                                        predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                        TableScan [TS_6] (rows=462000 width=1436)
+                                                          default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
+                                                <-Reducer 11 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_100]
+                                                    PartitionCols:_col1
+                                                    Merge Join Operator [MERGEJOIN_273] (rows=316788826 width=135)
+                                                      Conds:RS_97._col0=RS_98._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                                    <-Map 18 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_98]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_90] (rows=36524 width=1119)
+                                                          Output:["_col0"]
+                                                          Filter Operator [FIL_253] (rows=36524 width=1119)
+                                                            predicate:((d_year = 2002) and d_date_sk is not null)
+                                                            TableScan [TS_3] (rows=73049 width=1119)
+                                                              default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
+                                                    <-Map 1 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_97]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_87] (rows=287989836 width=135)
+                                                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                          Filter Operator [FIL_252] (rows=287989836 width=135)
+                                                            predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
+                                                            TableScan [TS_0] (rows=287989836 width=135)
+                                                              default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_order_number","cs_quantity","cs_ext_sales_price"]
+                                    <-Reducer 27 [CONTAINS]
+                                      Reduce Output Operator [RS_133]
+                                        PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5
+                                        Group By Operator [GBY_132] (rows=1149964734 width=103)
+                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:_col0, _col1, _col2, _col3, _col4, _col5
+                                          Select Operator [SEL_128] (rows=766650239 width=88)
+                                            Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                            Merge Join Operator [MERGEJOIN_278] (rows=766650239 width=88)
+                                              Conds:RS_125._col1, _col2=RS_126._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
+                                            <-Map 34 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_126]
+                                                PartitionCols:_col0, _col1
+                                                Select Operator [SEL_118] (rows=57591150 width=77)
+                                                  Output:["_col0","_col1","_col2","_col3"]
+                                                  Filter Operator [FIL_259] (rows=57591150 width=77)
+                                                    predicate:sr_item_sk is not null
+                                                    TableScan [TS_31] (rows=57591150 width=77)
+                                                      default@store_returns,store_returns,Tbl:COMPLETE,Col:NONE,Output:["sr_item_sk","sr_ticket_number","sr_return_quantity","sr_return_amt"]
+                                            <-Reducer 26 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_125]
+                                                PartitionCols:_col1, _col2
+                                                Merge Join Operator [MERGEJOIN_277] (rows=696954748 width=88)
+                                                  Conds:RS_122._col1=RS_123._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
+                                                <-Map 31 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_123]
+                                                    PartitionCols:_col0
+                                                    Select Operator [SEL_115] (rows=231000 width=1436)
+                                                      Output:["_col0","_col1","_col2","_col3","_col5"]
+                                                      Filter Operator [FIL_258] (rows=231000 width=1436)
+                                                        predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                         Please refer to the previous TableScan [TS_6]
+                                                <-Reducer 25 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_122]
+                                                    PartitionCols:_col1
+                                                    Merge Join Operator [MERGEJOIN_276] (rows=633595212 width=88)
+                                                      Conds:RS_119._col0=RS_120._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                                    <-Map 18 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_120]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_112] (rows=36524 width=1119)
+                                                          Output:["_col0"]
+                                                          Filter Operator [FIL_257] (rows=36524 width=1119)
+                                                            predicate:((d_year = 2002) and d_date_sk is not null)
+                                                             Please refer to the previous TableScan [TS_3]
+                                                    <-Map 33 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_119]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_109] (rows=575995635 width=88)
+                                                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                          Filter Operator [FIL_256] (rows=575995635 width=88)
+                                                            predicate:(ss_item_sk is not null and ss_sold_date_sk is not null)
+                                                            TableScan [TS_22] (rows=575995635 width=88)
+                                                              default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ticket_number","ss_quantity","ss_ext_sales_price"]
+                            <-Reducer 30 [CONTAINS]
+                              Reduce Output Operator [RS_162]
+                                PartitionCols:_col0, _col1, _col2, _col3
+                                Group By Operator [GBY_161] (rows=766649929 width=111)
+                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:_col0, _col1, _col2, _col3, _col4, _col5
+                                  Select Operator [SEL_157] (rows=191667562 width=135)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                    Merge Join Operator [MERGEJOIN_281] (rows=191667562 width=135)
+                                      Conds:RS_154._col1, _col2=RS_155._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
+                                    <-Map 36 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_155]
+                                        PartitionCols:_col0, _col1
+                                        Select Operator [SEL_147] (rows=14398467 width=92)
+                                          Output:["_col0","_col1","_col2","_col3"]
+                                          Filter Operator [FIL_263] (rows=14398467 width=92)
+                                            predicate:wr_item_sk is not null
+                                            TableScan [TS_60] (rows=14398467 width=92)
+                                              default@web_returns,web_returns,Tbl:COMPLETE,Col:NONE,Output:["wr_item_sk","wr_order_number","wr_return_quantity","wr_return_amt"]
+                                    <-Reducer 29 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_154]
+                                        PartitionCols:_col1, _col2
+                                        Merge Join Operator [MERGEJOIN_280] (rows=174243235 width=135)
+                                          Conds:RS_151._col1=RS_152._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
+                                        <-Map 31 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_152]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_144] (rows=231000 width=1436)
+                                              Output:["_col0","_col1","_col2","_col3","_col5"]
+                                              Filter Operator [FIL_262] (rows=231000 width=1436)
+                                                predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                 Please refer to the previous TableScan [TS_6]
+                                        <-Reducer 28 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_151]
+                                            PartitionCols:_col1
+                                            Merge Join Operator [MERGEJOIN_279] (rows=158402938 width=135)
+                                              Conds:RS_148._col0=RS_149._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                            <-Map 18 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_149]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_141] (rows=36524 width=1119)
+                                                  Output:["_col0"]
+                                                  Filter Operator [FIL_261] (rows=36524 width=1119)
+                                                    predicate:((d_year = 2002) and d_date_sk is not null)
+                                                     Please refer to the previous TableScan [TS_3]
+                                            <-Map 35 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_148]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_138] (rows=144002668 width=135)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                  Filter Operator [FIL_260] (rows=144002668 width=135)
+                                                    predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
+                                                    TableScan [TS_51] (rows=144002668 width=135)
+                                                      default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_order_number","ws_quantity","ws_ext_sales_price"]
+                    <-Reducer 8 [SIMPLE_EDGE]
+                      SHUFFLE [RS_170]
                         PartitionCols:_col0, _col1, _col2, _col3
-                        Group By Operator [GBY_72] (rows=670816148 width=108)
-                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
-                        <-Union 5 [SIMPLE_EDGE]
-                          <-Reducer 17 [CONTAINS]
-                            Reduce Output Operator [RS_71]
-                              PartitionCols:_col0, _col1, _col2, _col3
-                              Group By Operator [GBY_70] (rows=1341632296 width=108)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3
-                                Select Operator [SEL_43] (rows=766650239 width=88)
-                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                  Merge Join Operator [MERGEJOIN_247] (rows=766650239 width=88)
-                                    Conds:RS_40._col1, _col2=RS_41._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
-                                  <-Map 30 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_41]
-                                      PartitionCols:_col0, _col1
-                                      Select Operator [SEL_33] (rows=57591150 width=77)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_225] (rows=57591150 width=77)
-                                          predicate:sr_item_sk is not null
-                                           Please refer to the previous TableScan [TS_31]
-                                  <-Reducer 16 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_40]
-                                      PartitionCols:_col1, _col2
-                                      Merge Join Operator [MERGEJOIN_246] (rows=696954748 width=88)
-                                        Conds:RS_37._col1=RS_38._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
-                                      <-Map 27 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_38]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_30] (rows=231000 width=1436)
-                                            Output:["_col0","_col1","_col2","_col3","_col5"]
-                                            Filter Operator [FIL_224] (rows=231000 width=1436)
-                                              predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                               Please refer to the previous TableScan [TS_6]
-                                      <-Reducer 15 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_37]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_245] (rows=633595212 width=88)
-                                            Conds:RS_34._col0=RS_35._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                                          <-Map 14 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_35]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_27] (rows=36524 width=1119)
-                                                Output:["_col0"]
-                                                Filter Operator [FIL_223] (rows=36524 width=1119)
-                                                  predicate:((d_year = 2001) and d_date_sk is not null)
-                                                   Please refer to the previous TableScan [TS_3]
-                                          <-Map 29 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_34]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_24] (rows=575995635 width=88)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_222] (rows=575995635 width=88)
-                                                  predicate:(ss_item_sk is not null and ss_sold_date_sk is not null)
-                                                   Please refer to the previous TableScan [TS_22]
-                          <-Reducer 20 [CONTAINS]
-                            Reduce Output Operator [RS_71]
-                              PartitionCols:_col0, _col1, _col2, _col3
-                              Group By Operator [GBY_70] (rows=1341632296 width=108)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3
-                                Select Operator [SEL_67] (rows=191667562 width=135)
-                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                  Merge Join Operator [MERGEJOIN_250] (rows=191667562 width=135)
-                                    Conds:RS_64._col1, _col2=RS_65._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
-                                  <-Map 32 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_65]
-                                      PartitionCols:_col0, _col1
-                                      Select Operator [SEL_57] (rows=14398467 width=92)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_229] (rows=14398467 width=92)
-                                          predicate:wr_item_sk is not null
-                                           Please refer to the previous TableScan [TS_55]
-                                  <-Reducer 19 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_64]
-                                      PartitionCols:_col1, _col2
-                                      Merge Join Operator [MERGEJOIN_249] (rows=174243235 width=135)
-                                        Conds:RS_61._col1=RS_62._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
-                                      <-Map 27 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_62]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_54] (rows=231000 width=1436)
-                                            Output:["_col0","_col1","_col2","_col3","_col5"]
-                                            Filter Operator [FIL_228] (rows=231000 width=1436)
-                                              predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                               Please refer to the previous TableScan [TS_6]
-                                      <-Reducer 18 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_61]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_248] (rows=158402938 width=135)
-                                            Conds:RS_58._col0=RS_59._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                                          <-Map 14 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_59]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_51] (rows=36524 width=1119)
-                                                Output:["_col0"]
-                                                Filter Operator [FIL_227] (rows=36524 width=1119)
-                                                  predicate:((d_year = 2001) and d_date_sk is not null)
-                                                   Please refer to the previous TableScan [TS_3]
-                                          <-Map 31 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_58]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_48] (rows=144002668 width=135)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_226] (rows=144002668 width=135)
-                                                  predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
-                                                   Please refer to the previous TableScan [TS_46]
-                          <-Reducer 4 [CONTAINS]
-                            Reduce Output Operator [RS_71]
-                              PartitionCols:_col0, _col1, _col2, _col3
-                              Group By Operator [GBY_70] (rows=1341632296 width=108)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3
-                                Select Operator [SEL_21] (rows=383314495 width=135)
-                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                  Merge Join Operator [MERGEJOIN_244] (rows=383314495 width=135)
-                                    Conds:RS_18._col1, _col2=RS_19._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
-                                  <-Map 28 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_19]
-                                      PartitionCols:_col0, _col1
-                                      Select Operator [SEL_11] (rows=28798881 width=106)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_221] (rows=28798881 width=106)
-                                          predicate:cr_item_sk is not null
-                                           Please refer to the previous TableScan [TS_9]
-                                  <-Reducer 3 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_18]
-                                      PartitionCols:_col1, _col2
-                                      Merge Join Operator [MERGEJOIN_243] (rows=348467716 width=135)
-                                        Conds:RS_15._col1=RS_16._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
-                                      <-Map 27 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_16]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_8] (rows=231000 width=1436)
-                                            Output:["_col0","_col1","_col2","_col3","_col5"]
-                                            Filter Operator [FIL_220] (rows=231000 width=1436)
-                                              predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                               Please refer to the previous TableScan [TS_6]
-                                      <-Reducer 2 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_15]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_242] (rows=316788826 width=135)
-                                            Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                                          <-Map 14 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_13]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_5] (rows=36524 width=1119)
-                                                Output:["_col0"]
-                                                Filter Operator [FIL_219] (rows=36524 width=1119)
-                                                  predicate:((d_year = 2001) and d_date_sk is not null)
-                                                   Please refer to the previous TableScan [TS_3]
-                                          <-Map 1 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_12]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_2] (rows=287989836 width=135)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_218] (rows=287989836 width=135)
-                                                  predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
-                                                   Please refer to the previous TableScan [TS_0]
+                        Group By Operator [GBY_83] (rows=191662482 width=111)
+                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3
+                          Group By Operator [GBY_78] (rows=383324964 width=111)
+                            Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5
+                          <-Union 7 [SIMPLE_EDGE]
+                            <-Reducer 24 [CONTAINS]
+                              Reduce Output Operator [RS_77]
+                                PartitionCols:_col0, _col1, _col2, _col3
+                                Group By Operator [GBY_76] (rows=766649929 width=111)
+                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:_col0, _col1, _col2, _col3, _col4, _col5
+                                  Select Operator [SEL_72] (rows=191667562 width=135)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                    Merge Join Operator [MERGEJOIN_272] (rows=191667562 width=135)
+                                      Conds:RS_69._col1, _col2=RS_70._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
+                                    <-Map 36 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_70]
+                                        PartitionCols:_col0, _col1
+                                        Select Operator [SEL_62] (rows=14398467 width=92)
+                                          Output:["_col0","_col1","_col2","_col3"]
+                                          Filter Operator [FIL_251] (rows=14398467 width=92)
+                                            predicate:wr_item_sk is not null
+                                             Please refer to the previous TableScan [TS_60]
+                                    <-Reducer 23 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_69]
+                                        PartitionCols:_col1, _col2
+                                        Merge Join Operator [MERGEJOIN_271] (rows=174243235 width=135)
+                                          Conds:RS_66._col1=RS_67._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
+                                        <-Map 31 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_67]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_59] (rows=231000 width=1436)
+                                              Output:["_col0","_col1","_col2","_col3","_col5"]
+                                              Filter Operator [FIL_250] (rows=231000 width=1436)
+                                                predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                 Please refer to the previous TableScan [TS_6]
+                                        <-Reducer 22 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_66]
+                                            PartitionCols:_col1
+                                            Merge Join Operator [MERGEJOIN_270] (rows=158402938 width=135)
+                                              Conds:RS_63._col0=RS_64._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                            <-Map 18 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_64]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_56] (rows=36524 width=1119)
+                                                  Output:["_col0"]
+                                                  Filter Operator [FIL_249] (rows=36524 width=1119)
+                                                    predicate:((d_year = 2001) and d_date_sk is not null)
+                                                     Please refer to the previous TableScan [TS_3]
+                                            <-Map 35 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_63]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_53] (rows=144002668 width=135)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                  Filter Operator [FIL_248] (rows=144002668 width=135)
+                                                    predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
+                                                     Please refer to the previous TableScan [TS_51]
+                            <-Reducer 6 [CONTAINS]
+                              Reduce Output Operator [RS_77]
+                                PartitionCols:_col0, _col1, _col2, _col3
+                                Group By Operator [GBY_76] (rows=766649929 width=111)
+                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:_col0, _col1, _col2, _col3, _col4, _col5
+                                  Group By Operator [GBY_49] (rows=574982367 width=103)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5
+                                  <-Union 5 [SIMPLE_EDGE]
+                                    <-Reducer 21 [CONTAINS]
+                                      Reduce Output Operator [RS_48]
+                                        PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5
+                                        Group By Operator [GBY_47] (rows=1149964734 width=103)
+                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:_col0, _col1, _col2, _col3, _col4, _col5
+                                          Select Operator [SEL_43] (rows=766650239 width=88)
+                                            Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                            Merge Join Operator [MERGEJOIN_269] (rows=766650239 width=88)
+                                              Conds:RS_40._col1, _col2=RS_41._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
+                                            <-Map 34 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_41]
+                                                PartitionCols:_col0, _col1
+                                                Select Operator [SEL_33] (rows=57591150 width=77)
+                                                  Output:["_col0","_col1","_col2","_col3"]
+                                                  Filter Operator [FIL_247] (rows=57591150 width=77)
+                                                    predicate:sr_item_sk is not null
+                                                     Please refer to the previous TableScan [TS_31]
+                                            <-Reducer 20 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_40]
+                                                PartitionCols:_col1, _col2
+                                                Merge Join Operator [MERGEJOIN_268] (rows=696954748 width=88)
+                                                  Conds:RS_37._col1=RS_38._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
+                                                <-Map 31 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_38]
+                                                    PartitionCols:_col0
+                                                    Select Operator [SEL_30] (rows=231000 width=1436)
+                                                      Output:["_col0","_col1","_col2","_col3","_col5"]
+                                                      Filter Operator [FIL_246] (rows=231000 width=1436)
+                                                        predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                         Please refer to the previous TableScan [TS_6]
+                                                <-Reducer 19 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_37]
+                                                    PartitionCols:_col1
+                                                    Merge Join Operator [MERGEJOIN_267] (rows=633595212 width=88)
+                                                      Conds:RS_34._col0=RS_35._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                                    <-Map 18 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_35]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_27] (rows=36524 width=1119)
+                                                          Output:["_col0"]
+                                                          Filter Operator [FIL_245] (rows=36524 width=1119)
+                                                            predicate:((d_year = 2001) and d_date_sk is not null)
+                                                             Please refer to the previous TableScan [TS_3]
+                                                    <-Map 33 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_34]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_24] (rows=575995635 width=88)
+                                                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                          Filter Operator [FIL_244] (rows=575995635 width=88)
+                                                            predicate:(ss_item_sk is not null and ss_sold_date_sk is not null)
+                                                             Please refer to the previous TableScan [TS_22]
+                                    <-Reducer 4 [CONTAINS]
+                                      Reduce Output Operator [RS_48]
+                                        PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5
+                                        Group By Operator [GBY_47] (rows=1149964734 width=103)
+                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"],keys:_col0, _col1, _col2, _col3, _col4, _col5
+                                          Select Operator [SEL_21] (rows=383314495 width=135)
+                                            Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                            Merge Join Operator [MERGEJOIN_266] (rows=383314495 width=135)
+                                              Conds:RS_18._col1, _col2=RS_19._col0, _col1(Left Outer),Output:["_col3","_col4","_col8","_col9","_col10","_col12","_col15","_col16"]
+                                            <-Map 32 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_19]
+                                                PartitionCols:_col0, _col1
+                                                Select Operator [SEL_11] (rows=28798881 width=106)
+                                                  Output:["_col0","_col1","_col2","_col3"]
+                                                  Filter Operator [FIL_243] (rows=28798881 width=106)
+                                                    predicate:cr_item_sk is not null
+                                                     Please refer to the previous TableScan [TS_9]
+                                            <-Reducer 3 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_18]
+                                                PartitionCols:_col1, _col2
+                                                Merge Join Operator [MERGEJOIN_265] (rows=348467716 width=135)
+                                                  Conds:RS_15._col1=RS_16._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col8","_col9","_col10","_col12"]
+                                                <-Map 31 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_16]
+                                                    PartitionCols:_col0
+                                                    Select Operator [SEL_8] (rows=231000 width=1436)
+                                                      Output:["_col0","_col1","_col2","_col3","_col5"]
+                                                      Filter Operator [FIL_242] (rows=231000 width=1436)
+                                                        predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                         Please refer to the previous TableScan [TS_6]
+                                                <-Reducer 2 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_15]
+                                                    PartitionCols:_col1
+                                                    Merge Join Operator [MERGEJOIN_264] (rows=316788826 width=135)
+                                                      Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                                    <-Map 18 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_13]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_5] (rows=36524 width=1119)
+                                                          Output:["_col0"]
+                                                          Filter Operator [FIL_241] (rows=36524 width=1119)
+                                                            predicate:((d_year = 2001) and d_date_sk is not null)
+                                                             Please refer to the previous TableScan [TS_3]
+                                                    <-Map 1 [SIMPLE_EDGE]
+                                                      SHUFFLE [RS_12]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_2] (rows=287989836 width=135)
+                                                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                          Filter Operator [FIL_240] (rows=287989836 width=135)
+                                                            predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
+                                                             Please refer to the previous TableScan [TS_0]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query76.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query76.q.out b/ql/src/test/results/clientpositive/perf/query76.q.out
index 19243db..dcd5004 100644
--- a/ql/src/test/results/clientpositive/perf/query76.q.out
+++ b/ql/src/test/results/clientpositive/perf/query76.q.out
@@ -1,6 +1,48 @@
-PREHOOK: query: explain select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( SELECT 'store' as channel, 'ss_addr_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price FROM store_sales, item, date_dim WHERE ss_addr_sk IS NULL AND store_sales.ss_sold_date_sk=date_dim.d_date_sk AND store_sales.ss_item_sk=item.i_item_sk UNION ALL SELECT 'web' as channel, 'ws_web_page_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price FROM web_sales, item, date_dim WHERE ws_web_page_sk IS NULL AND web_sales.ws_sold_date_sk=date_dim.d_date_sk AND web_sales.ws_item_sk=item.i_item_sk UNION ALL SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price FROM catalog_sales, item, date_dim WHERE cs_warehouse_sk IS NULL AND catalog_sales.cs_sold_date_sk=date_dim.d_date_sk AND catalog_sales.cs_item_sk=item.i_item_sk) foo GROUP BY channel, col_name, d_
 year, d_qoy, i_category ORDER BY channel, col_name, d_year, d_qoy, i_category limit 100
+PREHOOK: query: explain
+select  channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM (
+        SELECT 'store' as channel, 'ss_addr_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price
+         FROM store_sales, item, date_dim
+         WHERE ss_addr_sk IS NULL
+           AND ss_sold_date_sk=d_date_sk
+           AND ss_item_sk=i_item_sk
+        UNION ALL
+        SELECT 'web' as channel, 'ws_web_page_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price
+         FROM web_sales, item, date_dim
+         WHERE ws_web_page_sk IS NULL
+           AND ws_sold_date_sk=d_date_sk
+           AND ws_item_sk=i_item_sk
+        UNION ALL
+        SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price
+         FROM catalog_sales, item, date_dim
+         WHERE cs_warehouse_sk IS NULL
+           AND cs_sold_date_sk=d_date_sk
+           AND cs_item_sk=i_item_sk) foo
+GROUP BY channel, col_name, d_year, d_qoy, i_category
+ORDER BY channel, col_name, d_year, d_qoy, i_category
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( SELECT 'store' as channel, 'ss_addr_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price FROM store_sales, item, date_dim WHERE ss_addr_sk IS NULL AND store_sales.ss_sold_date_sk=date_dim.d_date_sk AND store_sales.ss_item_sk=item.i_item_sk UNION ALL SELECT 'web' as channel, 'ws_web_page_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price FROM web_sales, item, date_dim WHERE ws_web_page_sk IS NULL AND web_sales.ws_sold_date_sk=date_dim.d_date_sk AND web_sales.ws_item_sk=item.i_item_sk UNION ALL SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price FROM catalog_sales, item, date_dim WHERE cs_warehouse_sk IS NULL AND catalog_sales.cs_sold_date_sk=date_dim.d_date_sk AND catalog_sales.cs_item_sk=item.i_item_sk) foo GROUP BY channel, col_name, d
 _year, d_qoy, i_category ORDER BY channel, col_name, d_year, d_qoy, i_category limit 100
+POSTHOOK: query: explain
+select  channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM (
+        SELECT 'store' as channel, 'ss_addr_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price
+         FROM store_sales, item, date_dim
+         WHERE ss_addr_sk IS NULL
+           AND ss_sold_date_sk=d_date_sk
+           AND ss_item_sk=i_item_sk
+        UNION ALL
+        SELECT 'web' as channel, 'ws_web_page_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price
+         FROM web_sales, item, date_dim
+         WHERE ws_web_page_sk IS NULL
+           AND ws_sold_date_sk=d_date_sk
+           AND ws_item_sk=i_item_sk
+        UNION ALL
+        SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price
+         FROM catalog_sales, item, date_dim
+         WHERE cs_warehouse_sk IS NULL
+           AND cs_sold_date_sk=d_date_sk
+           AND cs_item_sk=i_item_sk) foo
+GROUP BY channel, col_name, d_year, d_qoy, i_category
+ORDER BY channel, col_name, d_year, d_qoy, i_category
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 


[12/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query25.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query25.q.out b/ql/src/test/results/clientpositive/perf/query25.q.out
index 2b1c064..071e23e 100644
--- a/ql/src/test/results/clientpositive/perf/query25.q.out
+++ b/ql/src/test/results/clientpositive/perf/query25.q.out
@@ -1,6 +1,96 @@
-PREHOOK: query: explain select i_item_id ,i_item_desc ,s_store_id ,s_store_name ,sum(ss_net_profit) as store_sales_profit ,sum(sr_net_loss) as store_returns_loss ,sum(cs_net_profit) as catalog_sales_profit from store_sales ,store_returns ,catalog_sales ,date_dim d1 ,date_dim d2 ,date_dim d3 ,store ,item where d1.d_moy = 4 and d1.d_year = 1998 and d1.d_date_sk = ss_sold_date_sk and i_item_sk = ss_item_sk and s_store_sk = ss_store_sk and ss_customer_sk = sr_customer_sk and ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number and sr_returned_date_sk = d2.d_date_sk and d2.d_moy between 4 and 10 and d2.d_year = 1998 and sr_customer_sk = cs_bill_customer_sk and sr_item_sk = cs_item_sk and cs_sold_date_sk = d3.d_date_sk and d3.d_moy between 4 and 10 and d3.d_year = 1998 group by i_item_id ,i_item_desc ,s_store_id ,s_store_name order by i_item_id ,i_item_desc ,s_store_id ,s_store_name limit 100
+PREHOOK: query: explain
+select  
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ ,sum(ss_net_profit) as store_sales_profit
+ ,sum(sr_net_loss) as store_returns_loss
+ ,sum(cs_net_profit) as catalog_sales_profit
+ from
+ store_sales
+ ,store_returns
+ ,catalog_sales
+ ,date_dim d1
+ ,date_dim d2
+ ,date_dim d3
+ ,store
+ ,item
+ where
+ d1.d_moy = 4
+ and d1.d_year = 2000
+ and d1.d_date_sk = ss_sold_date_sk
+ and i_item_sk = ss_item_sk
+ and s_store_sk = ss_store_sk
+ and ss_customer_sk = sr_customer_sk
+ and ss_item_sk = sr_item_sk
+ and ss_ticket_number = sr_ticket_number
+ and sr_returned_date_sk = d2.d_date_sk
+ and d2.d_moy               between 4 and  10
+ and d2.d_year              = 2000
+ and sr_customer_sk = cs_bill_customer_sk
+ and sr_item_sk = cs_item_sk
+ and cs_sold_date_sk = d3.d_date_sk
+ and d3.d_moy               between 4 and  10 
+ and d3.d_year              = 2000
+ group by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ order by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select i_item_id ,i_item_desc ,s_store_id ,s_store_name ,sum(ss_net_profit) as store_sales_profit ,sum(sr_net_loss) as store_returns_loss ,sum(cs_net_profit) as catalog_sales_profit from store_sales ,store_returns ,catalog_sales ,date_dim d1 ,date_dim d2 ,date_dim d3 ,store ,item where d1.d_moy = 4 and d1.d_year = 1998 and d1.d_date_sk = ss_sold_date_sk and i_item_sk = ss_item_sk and s_store_sk = ss_store_sk and ss_customer_sk = sr_customer_sk and ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number and sr_returned_date_sk = d2.d_date_sk and d2.d_moy between 4 and 10 and d2.d_year = 1998 and sr_customer_sk = cs_bill_customer_sk and sr_item_sk = cs_item_sk and cs_sold_date_sk = d3.d_date_sk and d3.d_moy between 4 and 10 and d3.d_year = 1998 group by i_item_id ,i_item_desc ,s_store_id ,s_store_name order by i_item_id ,i_item_desc ,s_store_id ,s_store_name limit 100
+POSTHOOK: query: explain
+select  
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ ,sum(ss_net_profit) as store_sales_profit
+ ,sum(sr_net_loss) as store_returns_loss
+ ,sum(cs_net_profit) as catalog_sales_profit
+ from
+ store_sales
+ ,store_returns
+ ,catalog_sales
+ ,date_dim d1
+ ,date_dim d2
+ ,date_dim d3
+ ,store
+ ,item
+ where
+ d1.d_moy = 4
+ and d1.d_year = 2000
+ and d1.d_date_sk = ss_sold_date_sk
+ and i_item_sk = ss_item_sk
+ and s_store_sk = ss_store_sk
+ and ss_customer_sk = sr_customer_sk
+ and ss_item_sk = sr_item_sk
+ and ss_ticket_number = sr_ticket_number
+ and sr_returned_date_sk = d2.d_date_sk
+ and d2.d_moy               between 4 and  10
+ and d2.d_year              = 2000
+ and sr_customer_sk = cs_bill_customer_sk
+ and sr_item_sk = cs_item_sk
+ and cs_sold_date_sk = d3.d_date_sk
+ and d3.d_moy               between 4 and  10 
+ and d3.d_year              = 2000
+ group by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ order by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -77,7 +167,7 @@ Stage-0
                                       Select Operator [SEL_17] (rows=4058 width=1119)
                                         Output:["_col0"]
                                         Filter Operator [FIL_91] (rows=4058 width=1119)
-                                          predicate:(d_moy BETWEEN 4 AND 10 and (d_year = 1998) and d_date_sk is not null)
+                                          predicate:(d_moy BETWEEN 4 AND 10 and (d_year = 2000) and d_date_sk is not null)
                                           TableScan [TS_3] (rows=73049 width=1119)
                                             default@date_dim,d3,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
                                   <-Reducer 8 [SIMPLE_EDGE]
@@ -91,7 +181,7 @@ Stage-0
                                           Select Operator [SEL_14] (rows=18262 width=1119)
                                             Output:["_col0"]
                                             Filter Operator [FIL_90] (rows=18262 width=1119)
-                                              predicate:((d_moy = 4) and (d_year = 1998) and d_date_sk is not null)
+                                              predicate:((d_moy = 4) and (d_year = 2000) and d_date_sk is not null)
                                                Please refer to the previous TableScan [TS_3]
                                       <-Reducer 12 [SIMPLE_EDGE]
                                         SHUFFLE [RS_24]
@@ -127,7 +217,7 @@ Stage-0
                                 Select Operator [SEL_5] (rows=4058 width=1119)
                                   Output:["_col0"]
                                   Filter Operator [FIL_87] (rows=4058 width=1119)
-                                    predicate:(d_moy BETWEEN 4 AND 10 and (d_year = 1998) and d_date_sk is not null)
+                                    predicate:(d_moy BETWEEN 4 AND 10 and (d_year = 2000) and d_date_sk is not null)
                                      Please refer to the previous TableScan [TS_3]
                             <-Map 1 [SIMPLE_EDGE]
                               SHUFFLE [RS_37]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query26.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query26.q.out b/ql/src/test/results/clientpositive/perf/query26.q.out
index 1c94c40..125dc77 100644
--- a/ql/src/test/results/clientpositive/perf/query26.q.out
+++ b/ql/src/test/results/clientpositive/perf/query26.q.out
@@ -1,6 +1,42 @@
-PREHOOK: query: explain select i_item_id, avg(cs_quantity) agg1, avg(cs_list_price) agg2, avg(cs_coupon_amt) agg3, avg(cs_sales_price) agg4 from catalog_sales, customer_demographics, date_dim, item, promotion where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and catalog_sales.cs_item_sk = item.i_item_sk and catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk and catalog_sales.cs_promo_sk = promotion.p_promo_sk and cd_gender = 'F' and cd_marital_status = 'W' and cd_education_status = 'Primary' and (p_channel_email = 'N' or p_channel_event = 'N') and d_year = 1998 group by i_item_id order by i_item_id limit 100
+PREHOOK: query: explain
+select  i_item_id, 
+        avg(cs_quantity) agg1,
+        avg(cs_list_price) agg2,
+        avg(cs_coupon_amt) agg3,
+        avg(cs_sales_price) agg4 
+ from catalog_sales, customer_demographics, date_dim, item, promotion
+ where cs_sold_date_sk = d_date_sk and
+       cs_item_sk = i_item_sk and
+       cs_bill_cdemo_sk = cd_demo_sk and
+       cs_promo_sk = p_promo_sk and
+       cd_gender = 'F' and 
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       (p_channel_email = 'N' or p_channel_event = 'N') and
+       d_year = 1998 
+ group by i_item_id
+ order by i_item_id
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select i_item_id, avg(cs_quantity) agg1, avg(cs_list_price) agg2, avg(cs_coupon_amt) agg3, avg(cs_sales_price) agg4 from catalog_sales, customer_demographics, date_dim, item, promotion where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk and catalog_sales.cs_item_sk = item.i_item_sk and catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk and catalog_sales.cs_promo_sk = promotion.p_promo_sk and cd_gender = 'F' and cd_marital_status = 'W' and cd_education_status = 'Primary' and (p_channel_email = 'N' or p_channel_event = 'N') and d_year = 1998 group by i_item_id order by i_item_id limit 100
+POSTHOOK: query: explain
+select  i_item_id, 
+        avg(cs_quantity) agg1,
+        avg(cs_list_price) agg2,
+        avg(cs_coupon_amt) agg3,
+        avg(cs_sales_price) agg4 
+ from catalog_sales, customer_demographics, date_dim, item, promotion
+ where cs_sold_date_sk = d_date_sk and
+       cs_item_sk = i_item_sk and
+       cs_bill_cdemo_sk = cd_demo_sk and
+       cs_promo_sk = p_promo_sk and
+       cd_gender = 'F' and 
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       (p_channel_email = 'N' or p_channel_event = 'N') and
+       d_year = 1998 
+ group by i_item_id
+ order by i_item_id
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query28.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query28.q.out b/ql/src/test/results/clientpositive/perf/query28.q.out
index 78129cf..1fe7f15 100644
--- a/ql/src/test/results/clientpositive/perf/query28.q.out
+++ b/ql/src/test/results/clientpositive/perf/query28.q.out
@@ -1,5 +1,6 @@
 Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 3' is a cross product
-PREHOOK: query: explain select  *
+PREHOOK: query: explain
+select  *
 from (select avg(ss_list_price) B1_LP
             ,count(ss_list_price) B1_CNT
             ,count(distinct ss_list_price) B1_CNTD
@@ -50,7 +51,8 @@ from (select avg(ss_list_price) B1_LP
           or ss_wholesale_cost between 42 and 42+20)) B6
 limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select  *
+POSTHOOK: query: explain
+select  *
 from (select avg(ss_list_price) B1_LP
             ,count(ss_list_price) B1_CNT
             ,count(distinct ss_list_price) B1_CNTD

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query29.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query29.q.out b/ql/src/test/results/clientpositive/perf/query29.q.out
index ec303d3..07306c0 100644
--- a/ql/src/test/results/clientpositive/perf/query29.q.out
+++ b/ql/src/test/results/clientpositive/perf/query29.q.out
@@ -1,6 +1,94 @@
-PREHOOK: query: explain select i_item_id ,i_item_desc ,s_store_id ,s_store_name ,sum(ss_quantity) as store_sales_quantity ,sum(sr_return_quantity) as store_returns_quantity ,sum(cs_quantity) as catalog_sales_quantity from store_sales ,store_returns ,catalog_sales ,date_dim d1 ,date_dim d2 ,date_dim d3 ,store ,item where d1.d_moy = 2 and d1.d_year = 2000 and d1.d_date_sk = ss_sold_date_sk and i_item_sk = ss_item_sk and s_store_sk = ss_store_sk and ss_customer_sk = sr_customer_sk and ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number and sr_returned_date_sk = d2.d_date_sk and d2.d_moy between 2 and 2 + 3 and d2.d_year = 2000 and sr_customer_sk = cs_bill_customer_sk and sr_item_sk = cs_item_sk and cs_sold_date_sk = d3.d_date_sk and d3.d_year in (2000,2000+1,2000+2) group by i_item_id ,i_item_desc ,s_store_id ,s_store_name order by i_item_id ,i_item_desc ,s_store_id ,s_store_name limit 100
+PREHOOK: query: explain
+select   
+     i_item_id
+    ,i_item_desc
+    ,s_store_id
+    ,s_store_name
+    ,sum(ss_quantity)        as store_sales_quantity
+    ,sum(sr_return_quantity) as store_returns_quantity
+    ,sum(cs_quantity)        as catalog_sales_quantity
+ from
+    store_sales
+   ,store_returns
+   ,catalog_sales
+   ,date_dim             d1
+   ,date_dim             d2
+   ,date_dim             d3
+   ,store
+   ,item
+ where
+     d1.d_moy               = 4 
+ and d1.d_year              = 1999
+ and d1.d_date_sk           = ss_sold_date_sk
+ and i_item_sk              = ss_item_sk
+ and s_store_sk             = ss_store_sk
+ and ss_customer_sk         = sr_customer_sk
+ and ss_item_sk             = sr_item_sk
+ and ss_ticket_number       = sr_ticket_number
+ and sr_returned_date_sk    = d2.d_date_sk
+ and d2.d_moy               between 4 and  4 + 3 
+ and d2.d_year              = 1999
+ and sr_customer_sk         = cs_bill_customer_sk
+ and sr_item_sk             = cs_item_sk
+ and cs_sold_date_sk        = d3.d_date_sk     
+ and d3.d_year              in (1999,1999+1,1999+2)
+ group by
+    i_item_id
+   ,i_item_desc
+   ,s_store_id
+   ,s_store_name
+ order by
+    i_item_id 
+   ,i_item_desc
+   ,s_store_id
+   ,s_store_name
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select i_item_id ,i_item_desc ,s_store_id ,s_store_name ,sum(ss_quantity) as store_sales_quantity ,sum(sr_return_quantity) as store_returns_quantity ,sum(cs_quantity) as catalog_sales_quantity from store_sales ,store_returns ,catalog_sales ,date_dim d1 ,date_dim d2 ,date_dim d3 ,store ,item where d1.d_moy = 2 and d1.d_year = 2000 and d1.d_date_sk = ss_sold_date_sk and i_item_sk = ss_item_sk and s_store_sk = ss_store_sk and ss_customer_sk = sr_customer_sk and ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number and sr_returned_date_sk = d2.d_date_sk and d2.d_moy between 2 and 2 + 3 and d2.d_year = 2000 and sr_customer_sk = cs_bill_customer_sk and sr_item_sk = cs_item_sk and cs_sold_date_sk = d3.d_date_sk and d3.d_year in (2000,2000+1,2000+2) group by i_item_id ,i_item_desc ,s_store_id ,s_store_name order by i_item_id ,i_item_desc ,s_store_id ,s_store_name limit 100
+POSTHOOK: query: explain
+select   
+     i_item_id
+    ,i_item_desc
+    ,s_store_id
+    ,s_store_name
+    ,sum(ss_quantity)        as store_sales_quantity
+    ,sum(sr_return_quantity) as store_returns_quantity
+    ,sum(cs_quantity)        as catalog_sales_quantity
+ from
+    store_sales
+   ,store_returns
+   ,catalog_sales
+   ,date_dim             d1
+   ,date_dim             d2
+   ,date_dim             d3
+   ,store
+   ,item
+ where
+     d1.d_moy               = 4 
+ and d1.d_year              = 1999
+ and d1.d_date_sk           = ss_sold_date_sk
+ and i_item_sk              = ss_item_sk
+ and s_store_sk             = ss_store_sk
+ and ss_customer_sk         = sr_customer_sk
+ and ss_item_sk             = sr_item_sk
+ and ss_ticket_number       = sr_ticket_number
+ and sr_returned_date_sk    = d2.d_date_sk
+ and d2.d_moy               between 4 and  4 + 3 
+ and d2.d_year              = 1999
+ and sr_customer_sk         = cs_bill_customer_sk
+ and sr_item_sk             = cs_item_sk
+ and cs_sold_date_sk        = d3.d_date_sk     
+ and d3.d_year              in (1999,1999+1,1999+2)
+ group by
+    i_item_id
+   ,i_item_desc
+   ,s_store_id
+   ,s_store_name
+ order by
+    i_item_id 
+   ,i_item_desc
+   ,s_store_id
+   ,s_store_name
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -77,7 +165,7 @@ Stage-0
                                       Select Operator [SEL_17] (rows=4058 width=1119)
                                         Output:["_col0"]
                                         Filter Operator [FIL_91] (rows=4058 width=1119)
-                                          predicate:(d_moy BETWEEN 2 AND 5 and (d_year = 2000) and d_date_sk is not null)
+                                          predicate:(d_moy BETWEEN 4 AND 7 and (d_year = 1999) and d_date_sk is not null)
                                           TableScan [TS_12] (rows=73049 width=1119)
                                             default@date_dim,d1,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
                                   <-Reducer 10 [SIMPLE_EDGE]
@@ -91,7 +179,7 @@ Stage-0
                                           Select Operator [SEL_14] (rows=18262 width=1119)
                                             Output:["_col0"]
                                             Filter Operator [FIL_90] (rows=18262 width=1119)
-                                              predicate:((d_moy = 2) and (d_year = 2000) and d_date_sk is not null)
+                                              predicate:((d_moy = 4) and (d_year = 1999) and d_date_sk is not null)
                                                Please refer to the previous TableScan [TS_12]
                                       <-Reducer 9 [SIMPLE_EDGE]
                                         SHUFFLE [RS_24]
@@ -136,7 +224,7 @@ Stage-0
                                 Select Operator [SEL_5] (rows=36525 width=1119)
                                   Output:["_col0"]
                                   Filter Operator [FIL_87] (rows=36525 width=1119)
-                                    predicate:((d_year) IN (2000, 2001, 2002) and d_date_sk is not null)
+                                    predicate:((d_year) IN (1999, 2000, 2001) and d_date_sk is not null)
                                     TableScan [TS_3] (rows=73049 width=1119)
                                       default@date_dim,d3,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query3.q.out b/ql/src/test/results/clientpositive/perf/query3.q.out
index ba37c27..33bcb31 100644
--- a/ql/src/test/results/clientpositive/perf/query3.q.out
+++ b/ql/src/test/results/clientpositive/perf/query3.q.out
@@ -1,6 +1,42 @@
-PREHOOK: query: explain select dt.d_year ,item.i_brand_id brand_id ,item.i_brand brand ,sum(ss_ext_sales_price) sum_agg from date_dim dt ,store_sales ,item where dt.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and item.i_manufact_id = 436 and dt.d_moy=12 group by dt.d_year ,item.i_brand ,item.i_brand_id order by dt.d_year ,sum_agg desc ,brand_id limit 100
+PREHOOK: query: explain
+select  dt.d_year 
+       ,item.i_brand_id brand_id 
+       ,item.i_brand brand
+       ,sum(ss_ext_sales_price) sum_agg
+ from  date_dim dt 
+      ,store_sales
+      ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+   and store_sales.ss_item_sk = item.i_item_sk
+   and item.i_manufact_id = 436
+   and dt.d_moy=12
+ group by dt.d_year
+      ,item.i_brand
+      ,item.i_brand_id
+ order by dt.d_year
+         ,sum_agg desc
+         ,brand_id
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select dt.d_year ,item.i_brand_id brand_id ,item.i_brand brand ,sum(ss_ext_sales_price) sum_agg from date_dim dt ,store_sales ,item where dt.d_date_sk = store_sales.ss_sold_date_sk and store_sales.ss_item_sk = item.i_item_sk and item.i_manufact_id = 436 and dt.d_moy=12 group by dt.d_year ,item.i_brand ,item.i_brand_id order by dt.d_year ,sum_agg desc ,brand_id limit 100
+POSTHOOK: query: explain
+select  dt.d_year 
+       ,item.i_brand_id brand_id 
+       ,item.i_brand brand
+       ,sum(ss_ext_sales_price) sum_agg
+ from  date_dim dt 
+      ,store_sales
+      ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+   and store_sales.ss_item_sk = item.i_item_sk
+   and item.i_manufact_id = 436
+   and dt.d_moy=12
+ group by dt.d_year
+      ,item.i_brand
+      ,item.i_brand_id
+ order by dt.d_year
+         ,sum_agg desc
+         ,brand_id
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query30.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query30.q.out b/ql/src/test/results/clientpositive/perf/query30.q.out
index abe92b5..a86f11d 100644
--- a/ql/src/test/results/clientpositive/perf/query30.q.out
+++ b/ql/src/test/results/clientpositive/perf/query30.q.out
@@ -1,4 +1,5 @@
-PREHOOK: query: explain with customer_total_return as
+PREHOOK: query: explain
+with customer_total_return as
  (select wr_returning_customer_sk as ctr_customer_sk
         ,ca_state as ctr_state, 
  	sum(wr_return_amt) as ctr_total_return
@@ -27,7 +28,8 @@ PREHOOK: query: explain with customer_total_return as
                   ,c_last_review_date,ctr_total_return
 limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with customer_total_return as
+POSTHOOK: query: explain
+with customer_total_return as
  (select wr_returning_customer_sk as ctr_customer_sk
         ,ca_state as ctr_state, 
  	sum(wr_return_amt) as ctr_total_return

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query31.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query31.q.out b/ql/src/test/results/clientpositive/perf/query31.q.out
index b456244..b3a74f3 100644
--- a/ql/src/test/results/clientpositive/perf/query31.q.out
+++ b/ql/src/test/results/clientpositive/perf/query31.q.out
@@ -1,301 +1,394 @@
-PREHOOK: query: explain with ss as (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales from store_sales,date_dim,customer_address where ss_sold_date_sk = d_date_sk and ss_addr_sk=ca_address_sk group by ca_county,d_qoy, d_year), ws as (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales from web_sales,date_dim,customer_address where ws_sold_date_sk = d_date_sk and ws_bill_addr_sk=ca_address_sk group by ca_county,d_qoy, d_year) select ss1.ca_county ,ss1.d_year ,ws2.web_sales/ws1.web_sales web_q1_q2_increase ,ss2.store_sales/ss1.store_sales store_q1_q2_increase ,ws3.web_sales/ws2.web_sales web_q2_q3_increase ,ss3.store_sales/ss2.store_sales store_q2_q3_increase from ss ss1 ,ss ss2 ,ss ss3 ,ws ws1 ,ws ws2 ,ws ws3 where ss1.d_qoy = 1 and ss1.d_year = 1998 and ss1.ca_county = ss2.ca_county and ss2.d_qoy = 2 and ss2.d_year = 1998 and ss2.ca_county = ss3.ca_county and ss3.d_qoy = 3 and ss3.d_year = 1998 and ss1.ca_county = ws1.ca_county and ws1.d_qoy = 1 a
 nd ws1.d_year = 1998 and ws1.ca_county = ws2.ca_county and ws2.d_qoy = 2 and ws2.d_year = 1998 and ws1.ca_county = ws3.ca_county and ws3.d_qoy = 3 and ws3.d_year =1998 and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end order by web_q1_q2_increase
+PREHOOK: query: explain
+with ss as
+ (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales
+ from store_sales,date_dim,customer_address
+ where ss_sold_date_sk = d_date_sk
+  and ss_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year),
+ ws as
+ (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales
+ from web_sales,date_dim,customer_address
+ where ws_sold_date_sk = d_date_sk
+  and ws_bill_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year)
+ select /* tt */
+        ss1.ca_county
+       ,ss1.d_year
+       ,ws2.web_sales/ws1.web_sales web_q1_q2_increase
+       ,ss2.store_sales/ss1.store_sales store_q1_q2_increase
+       ,ws3.web_sales/ws2.web_sales web_q2_q3_increase
+       ,ss3.store_sales/ss2.store_sales store_q2_q3_increase
+ from
+        ss ss1
+       ,ss ss2
+       ,ss ss3
+       ,ws ws1
+       ,ws ws2
+       ,ws ws3
+ where
+    ss1.d_qoy = 1
+    and ss1.d_year = 2000
+    and ss1.ca_county = ss2.ca_county
+    and ss2.d_qoy = 2
+    and ss2.d_year = 2000
+ and ss2.ca_county = ss3.ca_county
+    and ss3.d_qoy = 3
+    and ss3.d_year = 2000
+    and ss1.ca_county = ws1.ca_county
+    and ws1.d_qoy = 1
+    and ws1.d_year = 2000
+    and ws1.ca_county = ws2.ca_county
+    and ws2.d_qoy = 2
+    and ws2.d_year = 2000
+    and ws1.ca_county = ws3.ca_county
+    and ws3.d_qoy = 3
+    and ws3.d_year =2000
+    and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end 
+       > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end
+    and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end
+       > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end
+ order by ss1.d_year
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with ss as (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales from store_sales,date_dim,customer_address where ss_sold_date_sk = d_date_sk and ss_addr_sk=ca_address_sk group by ca_county,d_qoy, d_year), ws as (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales from web_sales,date_dim,customer_address where ws_sold_date_sk = d_date_sk and ws_bill_addr_sk=ca_address_sk group by ca_county,d_qoy, d_year) select ss1.ca_county ,ss1.d_year ,ws2.web_sales/ws1.web_sales web_q1_q2_increase ,ss2.store_sales/ss1.store_sales store_q1_q2_increase ,ws3.web_sales/ws2.web_sales web_q2_q3_increase ,ss3.store_sales/ss2.store_sales store_q2_q3_increase from ss ss1 ,ss ss2 ,ss ss3 ,ws ws1 ,ws ws2 ,ws ws3 where ss1.d_qoy = 1 and ss1.d_year = 1998 and ss1.ca_county = ss2.ca_county and ss2.d_qoy = 2 and ss2.d_year = 1998 and ss2.ca_county = ss3.ca_county and ss3.d_qoy = 3 and ss3.d_year = 1998 and ss1.ca_county = ws1.ca_county and ws1.d_qoy = 1 
 and ws1.d_year = 1998 and ws1.ca_county = ws2.ca_county and ws2.d_qoy = 2 and ws2.d_year = 1998 and ws1.ca_county = ws3.ca_county and ws3.d_qoy = 3 and ws3.d_year =1998 and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end order by web_q1_q2_increase
+POSTHOOK: query: explain
+with ss as
+ (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales
+ from store_sales,date_dim,customer_address
+ where ss_sold_date_sk = d_date_sk
+  and ss_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year),
+ ws as
+ (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales
+ from web_sales,date_dim,customer_address
+ where ws_sold_date_sk = d_date_sk
+  and ws_bill_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year)
+ select /* tt */
+        ss1.ca_county
+       ,ss1.d_year
+       ,ws2.web_sales/ws1.web_sales web_q1_q2_increase
+       ,ss2.store_sales/ss1.store_sales store_q1_q2_increase
+       ,ws3.web_sales/ws2.web_sales web_q2_q3_increase
+       ,ss3.store_sales/ss2.store_sales store_q2_q3_increase
+ from
+        ss ss1
+       ,ss ss2
+       ,ss ss3
+       ,ws ws1
+       ,ws ws2
+       ,ws ws3
+ where
+    ss1.d_qoy = 1
+    and ss1.d_year = 2000
+    and ss1.ca_county = ss2.ca_county
+    and ss2.d_qoy = 2
+    and ss2.d_year = 2000
+ and ss2.ca_county = ss3.ca_county
+    and ss3.d_qoy = 3
+    and ss3.d_year = 2000
+    and ss1.ca_county = ws1.ca_county
+    and ws1.d_qoy = 1
+    and ws1.d_year = 2000
+    and ws1.ca_county = ws2.ca_county
+    and ws2.d_qoy = 2
+    and ws2.d_year = 2000
+    and ws1.ca_county = ws3.ca_county
+    and ws3.d_qoy = 3
+    and ws3.d_year =2000
+    and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end 
+       > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end
+    and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end
+       > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end
+ order by ss1.d_year
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 10 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE)
-Reducer 11 <- Map 24 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE)
-Reducer 12 <- Reducer 11 (SIMPLE_EDGE)
-Reducer 14 <- Map 13 (SIMPLE_EDGE), Map 25 (SIMPLE_EDGE)
-Reducer 15 <- Map 24 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE)
-Reducer 16 <- Reducer 15 (SIMPLE_EDGE)
-Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE)
-Reducer 18 <- Map 13 (SIMPLE_EDGE), Map 25 (SIMPLE_EDGE)
-Reducer 19 <- Map 24 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE)
-Reducer 20 <- Reducer 19 (SIMPLE_EDGE)
-Reducer 21 <- Map 13 (SIMPLE_EDGE), Map 25 (SIMPLE_EDGE)
-Reducer 22 <- Map 24 (SIMPLE_EDGE), Reducer 21 (SIMPLE_EDGE)
-Reducer 23 <- Reducer 22 (SIMPLE_EDGE)
-Reducer 3 <- Map 24 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 10 <- Map 23 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+Reducer 11 <- Reducer 10 (SIMPLE_EDGE)
+Reducer 13 <- Map 12 (SIMPLE_EDGE), Map 24 (SIMPLE_EDGE)
+Reducer 14 <- Map 23 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE)
+Reducer 15 <- Reducer 14 (SIMPLE_EDGE)
+Reducer 16 <- Reducer 15 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE)
+Reducer 17 <- Map 12 (SIMPLE_EDGE), Map 24 (SIMPLE_EDGE)
+Reducer 18 <- Map 23 (SIMPLE_EDGE), Reducer 17 (SIMPLE_EDGE)
+Reducer 19 <- Reducer 18 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE)
+Reducer 20 <- Map 12 (SIMPLE_EDGE), Map 24 (SIMPLE_EDGE)
+Reducer 21 <- Map 23 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE)
+Reducer 22 <- Reducer 21 (SIMPLE_EDGE)
+Reducer 3 <- Map 23 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 12 (SIMPLE_EDGE), Reducer 17 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
-Reducer 7 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE)
-Reducer 8 <- Map 24 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 11 (SIMPLE_EDGE), Reducer 16 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE)
+Reducer 7 <- Map 23 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+Reducer 9 <- Map 1 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 6
-      File Output Operator [FS_136]
-        Select Operator [SEL_134] (rows=287493839 width=88)
+      Reducer 5
+      File Output Operator [FS_133]
+        Select Operator [SEL_132] (rows=287493839 width=88)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-        <-Reducer 5 [SIMPLE_EDGE]
-          SHUFFLE [RS_133]
-            Select Operator [SEL_132] (rows=287493839 width=88)
-              Output:["_col0","_col1","_col2","_col3","_col4"]
-              Filter Operator [FIL_130] (rows=287493839 width=88)
-                predicate:(CASE WHEN ((_col1 > 0)) THEN (CASE WHEN ((_col9 > 0)) THEN (((_col11 / _col9) > (_col5 / _col1))) ELSE ((null > (_col5 / _col1))) END) ELSE (CASE WHEN ((_col9 > 0)) THEN (((_col11 / _col9) > null)) ELSE (null) END) END and CASE WHEN ((_col3 > 0)) THEN (CASE WHEN ((_col7 > 0)) THEN (((_col9 / _col7) > (_col1 / _col3))) ELSE ((null > (_col1 / _col3))) END) ELSE (CASE WHEN ((_col7 > 0)) THEN (((_col9 / _col7) > null)) ELSE (null) END) END)
-                Merge Join Operator [MERGEJOIN_270] (rows=1149975359 width=88)
-                  Conds:RS_125._col0=RS_126._col0(Inner),RS_125._col0=RS_127._col0(Inner),RS_125._col0=RS_128._col0(Inner),Output:["_col0","_col1","_col3","_col5","_col7","_col9","_col11"]
-                <-Reducer 12 [SIMPLE_EDGE]
-                  SHUFFLE [RS_127]
+          Filter Operator [FIL_130] (rows=287493839 width=88)
+            predicate:(CASE WHEN ((_col1 > 0)) THEN (CASE WHEN ((_col9 > 0)) THEN (((_col11 / _col9) > (_col5 / _col1))) ELSE ((null > (_col5 / _col1))) END) ELSE (CASE WHEN ((_col9 > 0)) THEN (((_col11 / _col9) > null)) ELSE (null) END) END and CASE WHEN ((_col3 > 0)) THEN (CASE WHEN ((_col7 > 0)) THEN (((_col9 / _col7) > (_col1 / _col3))) ELSE ((null > (_col1 / _col3))) END) ELSE (CASE WHEN ((_col7 > 0)) THEN (((_col9 / _col7) > null)) ELSE (null) END) END)
+            Merge Join Operator [MERGEJOIN_267] (rows=1149975359 width=88)
+              Conds:RS_125._col0=RS_126._col0(Inner),RS_125._col0=RS_127._col0(Inner),RS_125._col0=RS_128._col0(Inner),Output:["_col0","_col1","_col3","_col5","_col7","_col9","_col11"]
+            <-Reducer 11 [SIMPLE_EDGE]
+              SHUFFLE [RS_127]
+                PartitionCols:_col0
+                Group By Operator [GBY_58] (rows=348477374 width=88)
+                  Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
+                <-Reducer 10 [SIMPLE_EDGE]
+                  SHUFFLE [RS_57]
                     PartitionCols:_col0
-                    Group By Operator [GBY_58] (rows=348477374 width=88)
+                    Group By Operator [GBY_56] (rows=696954748 width=88)
+                      Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7
+                      Merge Join Operator [MERGEJOIN_259] (rows=696954748 width=88)
+                        Conds:RS_52._col1=RS_53._col0(Inner),Output:["_col2","_col7"]
+                      <-Map 23 [SIMPLE_EDGE]
+                        SHUFFLE [RS_53]
+                          PartitionCols:_col0
+                          Select Operator [SEL_48] (rows=40000000 width=1014)
+                            Output:["_col0","_col1"]
+                            Filter Operator [FIL_244] (rows=40000000 width=1014)
+                              predicate:(ca_address_sk is not null and ca_county is not null)
+                              TableScan [TS_6] (rows=40000000 width=1014)
+                                default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"]
+                      <-Reducer 9 [SIMPLE_EDGE]
+                        SHUFFLE [RS_52]
+                          PartitionCols:_col1
+                          Merge Join Operator [MERGEJOIN_258] (rows=633595212 width=88)
+                            Conds:RS_49._col0=RS_50._col0(Inner),Output:["_col1","_col2"]
+                          <-Map 12 [SIMPLE_EDGE]
+                            SHUFFLE [RS_50]
+                              PartitionCols:_col0
+                              Select Operator [SEL_45] (rows=18262 width=1119)
+                                Output:["_col0"]
+                                Filter Operator [FIL_243] (rows=18262 width=1119)
+                                  predicate:((d_qoy = 3) and (d_year = 2000) and d_date_sk is not null)
+                                  TableScan [TS_3] (rows=73049 width=1119)
+                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"]
+                          <-Map 1 [SIMPLE_EDGE]
+                            SHUFFLE [RS_49]
+                              PartitionCols:_col0
+                              Select Operator [SEL_42] (rows=575995635 width=88)
+                                Output:["_col0","_col1","_col2"]
+                                Filter Operator [FIL_242] (rows=575995635 width=88)
+                                  predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null)
+                                  TableScan [TS_0] (rows=575995635 width=88)
+                                    default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_addr_sk","ss_ext_sales_price"]
+            <-Reducer 16 [SIMPLE_EDGE]
+              SHUFFLE [RS_128]
+                PartitionCols:_col0
+                Merge Join Operator [MERGEJOIN_266] (rows=191667561 width=135)
+                  Conds:RS_120._col0=RS_121._col0(Inner),RS_120._col0=RS_122._col0(Inner),Output:["_col0","_col1","_col3","_col5"]
+                <-Reducer 15 [SIMPLE_EDGE]
+                  SHUFFLE [RS_120]
+                    PartitionCols:_col0
+                    Group By Operator [GBY_78] (rows=87121617 width=135)
                       Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                    <-Reducer 11 [SIMPLE_EDGE]
-                      SHUFFLE [RS_57]
+                    <-Reducer 14 [SIMPLE_EDGE]
+                      SHUFFLE [RS_77]
                         PartitionCols:_col0
-                        Group By Operator [GBY_56] (rows=696954748 width=88)
+                        Group By Operator [GBY_76] (rows=174243235 width=135)
                           Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7
-                          Merge Join Operator [MERGEJOIN_262] (rows=696954748 width=88)
-                            Conds:RS_52._col1=RS_53._col0(Inner),Output:["_col2","_col7"]
-                          <-Map 24 [SIMPLE_EDGE]
-                            SHUFFLE [RS_53]
+                          Merge Join Operator [MERGEJOIN_261] (rows=174243235 width=135)
+                            Conds:RS_72._col1=RS_73._col0(Inner),Output:["_col2","_col7"]
+                          <-Map 23 [SIMPLE_EDGE]
+                            SHUFFLE [RS_73]
                               PartitionCols:_col0
-                              Select Operator [SEL_48] (rows=40000000 width=1014)
+                              Select Operator [SEL_68] (rows=40000000 width=1014)
                                 Output:["_col0","_col1"]
                                 Filter Operator [FIL_247] (rows=40000000 width=1014)
                                   predicate:(ca_address_sk is not null and ca_county is not null)
-                                  TableScan [TS_6] (rows=40000000 width=1014)
-                                    default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"]
-                          <-Reducer 10 [SIMPLE_EDGE]
-                            SHUFFLE [RS_52]
+                                   Please refer to the previous TableScan [TS_6]
+                          <-Reducer 13 [SIMPLE_EDGE]
+                            SHUFFLE [RS_72]
                               PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_261] (rows=633595212 width=88)
-                                Conds:RS_49._col0=RS_50._col0(Inner),Output:["_col1","_col2"]
-                              <-Map 13 [SIMPLE_EDGE]
-                                SHUFFLE [RS_50]
+                              Merge Join Operator [MERGEJOIN_260] (rows=158402938 width=135)
+                                Conds:RS_69._col0=RS_70._col0(Inner),Output:["_col1","_col2"]
+                              <-Map 12 [SIMPLE_EDGE]
+                                SHUFFLE [RS_70]
                                   PartitionCols:_col0
-                                  Select Operator [SEL_45] (rows=18262 width=1119)
+                                  Select Operator [SEL_65] (rows=18262 width=1119)
                                     Output:["_col0"]
                                     Filter Operator [FIL_246] (rows=18262 width=1119)
-                                      predicate:((d_qoy = 3) and (d_year = 1998) and d_date_sk is not null)
-                                      TableScan [TS_3] (rows=73049 width=1119)
-                                        default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"]
-                              <-Map 1 [SIMPLE_EDGE]
-                                SHUFFLE [RS_49]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_42] (rows=575995635 width=88)
-                                    Output:["_col0","_col1","_col2"]
-                                    Filter Operator [FIL_245] (rows=575995635 width=88)
-                                      predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null)
-                                      TableScan [TS_0] (rows=575995635 width=88)
-                                        default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_addr_sk","ss_ext_sales_price"]
-                <-Reducer 17 [SIMPLE_EDGE]
-                  SHUFFLE [RS_128]
-                    PartitionCols:_col0
-                    Merge Join Operator [MERGEJOIN_269] (rows=191667561 width=135)
-                      Conds:RS_120._col0=RS_121._col0(Inner),RS_120._col0=RS_122._col0(Inner),Output:["_col0","_col1","_col3","_col5"]
-                    <-Reducer 16 [SIMPLE_EDGE]
-                      SHUFFLE [RS_120]
-                        PartitionCols:_col0
-                        Group By Operator [GBY_78] (rows=87121617 width=135)
-                          Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                        <-Reducer 15 [SIMPLE_EDGE]
-                          SHUFFLE [RS_77]
-                            PartitionCols:_col0
-                            Group By Operator [GBY_76] (rows=174243235 width=135)
-                              Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7
-                              Merge Join Operator [MERGEJOIN_264] (rows=174243235 width=135)
-                                Conds:RS_72._col1=RS_73._col0(Inner),Output:["_col2","_col7"]
-                              <-Map 24 [SIMPLE_EDGE]
-                                SHUFFLE [RS_73]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_68] (rows=40000000 width=1014)
-                                    Output:["_col0","_col1"]
-                                    Filter Operator [FIL_250] (rows=40000000 width=1014)
-                                      predicate:(ca_address_sk is not null and ca_county is not null)
-                                       Please refer to the previous TableScan [TS_6]
-                              <-Reducer 14 [SIMPLE_EDGE]
-                                SHUFFLE [RS_72]
-                                  PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_263] (rows=158402938 width=135)
-                                    Conds:RS_69._col0=RS_70._col0(Inner),Output:["_col1","_col2"]
-                                  <-Map 13 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_70]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_65] (rows=18262 width=1119)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_249] (rows=18262 width=1119)
-                                          predicate:((d_qoy = 1) and (d_year = 1998) and d_date_sk is not null)
-                                           Please refer to the previous TableScan [TS_3]
-                                  <-Map 25 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_69]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_62] (rows=144002668 width=135)
-                                        Output:["_col0","_col1","_col2"]
-                                        Filter Operator [FIL_248] (rows=144002668 width=135)
-                                          predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null)
-                                          TableScan [TS_60] (rows=144002668 width=135)
-                                            default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_bill_addr_sk","ws_ext_sales_price"]
-                    <-Reducer 20 [SIMPLE_EDGE]
-                      SHUFFLE [RS_121]
-                        PartitionCols:_col0
-                        Group By Operator [GBY_98] (rows=87121617 width=135)
-                          Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                        <-Reducer 19 [SIMPLE_EDGE]
-                          SHUFFLE [RS_97]
-                            PartitionCols:_col0
-                            Group By Operator [GBY_96] (rows=174243235 width=135)
-                              Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7
-                              Merge Join Operator [MERGEJOIN_266] (rows=174243235 width=135)
-                                Conds:RS_92._col1=RS_93._col0(Inner),Output:["_col2","_col7"]
-                              <-Map 24 [SIMPLE_EDGE]
-                                SHUFFLE [RS_93]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_88] (rows=40000000 width=1014)
-                                    Output:["_col0","_col1"]
-                                    Filter Operator [FIL_253] (rows=40000000 width=1014)
-                                      predicate:(ca_address_sk is not null and ca_county is not null)
-                                       Please refer to the previous TableScan [TS_6]
-                              <-Reducer 18 [SIMPLE_EDGE]
-                                SHUFFLE [RS_92]
-                                  PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_265] (rows=158402938 width=135)
-                                    Conds:RS_89._col0=RS_90._col0(Inner),Output:["_col1","_col2"]
-                                  <-Map 13 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_90]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_85] (rows=18262 width=1119)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_252] (rows=18262 width=1119)
-                                          predicate:((d_qoy = 2) and (d_year = 1998) and d_date_sk is not null)
-                                           Please refer to the previous TableScan [TS_3]
-                                  <-Map 25 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_89]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_82] (rows=144002668 width=135)
-                                        Output:["_col0","_col1","_col2"]
-                                        Filter Operator [FIL_251] (rows=144002668 width=135)
-                                          predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null)
-                                           Please refer to the previous TableScan [TS_60]
-                    <-Reducer 23 [SIMPLE_EDGE]
-                      SHUFFLE [RS_122]
-                        PartitionCols:_col0
-                        Group By Operator [GBY_118] (rows=87121617 width=135)
-                          Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                        <-Reducer 22 [SIMPLE_EDGE]
-                          SHUFFLE [RS_117]
-                            PartitionCols:_col0
-                            Group By Operator [GBY_116] (rows=174243235 width=135)
-                              Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7
-                              Merge Join Operator [MERGEJOIN_268] (rows=174243235 width=135)
-                                Conds:RS_112._col1=RS_113._col0(Inner),Output:["_col2","_col7"]
+                                      predicate:((d_qoy = 1) and (d_year = 2000) and d_date_sk is not null)
+                                       Please refer to the previous TableScan [TS_3]
                               <-Map 24 [SIMPLE_EDGE]
-                                SHUFFLE [RS_113]
+                                SHUFFLE [RS_69]
                                   PartitionCols:_col0
-                                  Select Operator [SEL_108] (rows=40000000 width=1014)
-                                    Output:["_col0","_col1"]
-                                    Filter Operator [FIL_256] (rows=40000000 width=1014)
-                                      predicate:(ca_address_sk is not null and ca_county is not null)
-                                       Please refer to the previous TableScan [TS_6]
-                              <-Reducer 21 [SIMPLE_EDGE]
-                                SHUFFLE [RS_112]
-                                  PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_267] (rows=158402938 width=135)
-                                    Conds:RS_109._col0=RS_110._col0(Inner),Output:["_col1","_col2"]
-                                  <-Map 13 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_110]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_105] (rows=18262 width=1119)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_255] (rows=18262 width=1119)
-                                          predicate:((d_qoy = 3) and (d_year = 1998) and d_date_sk is not null)
-                                           Please refer to the previous TableScan [TS_3]
-                                  <-Map 25 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_109]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_102] (rows=144002668 width=135)
-                                        Output:["_col0","_col1","_col2"]
-                                        Filter Operator [FIL_254] (rows=144002668 width=135)
-                                          predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null)
-                                           Please refer to the previous TableScan [TS_60]
-                <-Reducer 4 [SIMPLE_EDGE]
-                  SHUFFLE [RS_125]
+                                  Select Operator [SEL_62] (rows=144002668 width=135)
+                                    Output:["_col0","_col1","_col2"]
+                                    Filter Operator [FIL_245] (rows=144002668 width=135)
+                                      predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null)
+                                      TableScan [TS_60] (rows=144002668 width=135)
+                                        default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_bill_addr_sk","ws_ext_sales_price"]
+                <-Reducer 19 [SIMPLE_EDGE]
+                  SHUFFLE [RS_121]
                     PartitionCols:_col0
-                    Group By Operator [GBY_18] (rows=348477374 width=88)
+                    Group By Operator [GBY_98] (rows=87121617 width=135)
                       Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                    <-Reducer 3 [SIMPLE_EDGE]
-                      SHUFFLE [RS_17]
+                    <-Reducer 18 [SIMPLE_EDGE]
+                      SHUFFLE [RS_97]
                         PartitionCols:_col0
-                        Group By Operator [GBY_16] (rows=696954748 width=88)
+                        Group By Operator [GBY_96] (rows=174243235 width=135)
                           Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7
-                          Merge Join Operator [MERGEJOIN_258] (rows=696954748 width=88)
-                            Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col2","_col7"]
-                          <-Map 24 [SIMPLE_EDGE]
-                            SHUFFLE [RS_13]
+                          Merge Join Operator [MERGEJOIN_263] (rows=174243235 width=135)
+                            Conds:RS_92._col1=RS_93._col0(Inner),Output:["_col2","_col7"]
+                          <-Map 23 [SIMPLE_EDGE]
+                            SHUFFLE [RS_93]
                               PartitionCols:_col0
-                              Select Operator [SEL_8] (rows=40000000 width=1014)
+                              Select Operator [SEL_88] (rows=40000000 width=1014)
                                 Output:["_col0","_col1"]
-                                Filter Operator [FIL_241] (rows=40000000 width=1014)
+                                Filter Operator [FIL_250] (rows=40000000 width=1014)
                                   predicate:(ca_address_sk is not null and ca_county is not null)
                                    Please refer to the previous TableScan [TS_6]
-                          <-Reducer 2 [SIMPLE_EDGE]
-                            SHUFFLE [RS_12]
+                          <-Reducer 17 [SIMPLE_EDGE]
+                            SHUFFLE [RS_92]
                               PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_257] (rows=633595212 width=88)
-                                Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col1","_col2"]
-                              <-Map 13 [SIMPLE_EDGE]
-                                SHUFFLE [RS_10]
+                              Merge Join Operator [MERGEJOIN_262] (rows=158402938 width=135)
+                                Conds:RS_89._col0=RS_90._col0(Inner),Output:["_col1","_col2"]
+                              <-Map 12 [SIMPLE_EDGE]
+                                SHUFFLE [RS_90]
                                   PartitionCols:_col0
-                                  Select Operator [SEL_5] (rows=18262 width=1119)
+                                  Select Operator [SEL_85] (rows=18262 width=1119)
                                     Output:["_col0"]
-                                    Filter Operator [FIL_240] (rows=18262 width=1119)
-                                      predicate:((d_qoy = 2) and (d_year = 1998) and d_date_sk is not null)
+                                    Filter Operator [FIL_249] (rows=18262 width=1119)
+                                      predicate:((d_qoy = 2) and (d_year = 2000) and d_date_sk is not null)
                                        Please refer to the previous TableScan [TS_3]
-                              <-Map 1 [SIMPLE_EDGE]
-                                SHUFFLE [RS_9]
+                              <-Map 24 [SIMPLE_EDGE]
+                                SHUFFLE [RS_89]
                                   PartitionCols:_col0
-                                  Select Operator [SEL_2] (rows=575995635 width=88)
+                                  Select Operator [SEL_82] (rows=144002668 width=135)
                                     Output:["_col0","_col1","_col2"]
-                                    Filter Operator [FIL_239] (rows=575995635 width=88)
-                                      predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null)
-                                       Please refer to the previous TableScan [TS_0]
-                <-Reducer 9 [SIMPLE_EDGE]
-                  SHUFFLE [RS_126]
+                                    Filter Operator [FIL_248] (rows=144002668 width=135)
+                                      predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null)
+                                       Please refer to the previous TableScan [TS_60]
+                <-Reducer 22 [SIMPLE_EDGE]
+                  SHUFFLE [RS_122]
                     PartitionCols:_col0
-                    Group By Operator [GBY_38] (rows=348477374 width=88)
+                    Group By Operator [GBY_118] (rows=87121617 width=135)
                       Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                    <-Reducer 8 [SIMPLE_EDGE]
-                      SHUFFLE [RS_37]
+                    <-Reducer 21 [SIMPLE_EDGE]
+                      SHUFFLE [RS_117]
                         PartitionCols:_col0
-                        Group By Operator [GBY_36] (rows=696954748 width=88)
+                        Group By Operator [GBY_116] (rows=174243235 width=135)
                           Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7
-                          Merge Join Operator [MERGEJOIN_260] (rows=696954748 width=88)
-                            Conds:RS_32._col1=RS_33._col0(Inner),Output:["_col2","_col7"]
-                          <-Map 24 [SIMPLE_EDGE]
-                            SHUFFLE [RS_33]
+                          Merge Join Operator [MERGEJOIN_265] (rows=174243235 width=135)
+                            Conds:RS_112._col1=RS_113._col0(Inner),Output:["_col2","_col7"]
+                          <-Map 23 [SIMPLE_EDGE]
+                            SHUFFLE [RS_113]
                               PartitionCols:_col0
-                              Select Operator [SEL_28] (rows=40000000 width=1014)
+                              Select Operator [SEL_108] (rows=40000000 width=1014)
                                 Output:["_col0","_col1"]
-                                Filter Operator [FIL_244] (rows=40000000 width=1014)
+                                Filter Operator [FIL_253] (rows=40000000 width=1014)
                                   predicate:(ca_address_sk is not null and ca_county is not null)
                                    Please refer to the previous TableScan [TS_6]
-                          <-Reducer 7 [SIMPLE_EDGE]
-                            SHUFFLE [RS_32]
+                          <-Reducer 20 [SIMPLE_EDGE]
+                            SHUFFLE [RS_112]
                               PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_259] (rows=633595212 width=88)
-                                Conds:RS_29._col0=RS_30._col0(Inner),Output:["_col1","_col2"]
-                              <-Map 13 [SIMPLE_EDGE]
-                                SHUFFLE [RS_30]
+                              Merge Join Operator [MERGEJOIN_264] (rows=158402938 width=135)
+                                Conds:RS_109._col0=RS_110._col0(Inner),Output:["_col1","_col2"]
+                              <-Map 12 [SIMPLE_EDGE]
+                                SHUFFLE [RS_110]
                                   PartitionCols:_col0
-                                  Select Operator [SEL_25] (rows=18262 width=1119)
+                                  Select Operator [SEL_105] (rows=18262 width=1119)
                                     Output:["_col0"]
-                                    Filter Operator [FIL_243] (rows=18262 width=1119)
-                                      predicate:((d_qoy = 1) and (d_year = 1998) and d_date_sk is not null)
+                                    Filter Operator [FIL_252] (rows=18262 width=1119)
+                                      predicate:((d_qoy = 3) and (d_year = 2000) and d_date_sk is not null)
                                        Please refer to the previous TableScan [TS_3]
-                              <-Map 1 [SIMPLE_EDGE]
-                                SHUFFLE [RS_29]
+                              <-Map 24 [SIMPLE_EDGE]
+                                SHUFFLE [RS_109]
                                   PartitionCols:_col0
-                                  Select Operator [SEL_22] (rows=575995635 width=88)
+                                  Select Operator [SEL_102] (rows=144002668 width=135)
                                     Output:["_col0","_col1","_col2"]
-                                    Filter Operator [FIL_242] (rows=575995635 width=88)
-                                      predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null)
-                                       Please refer to the previous TableScan [TS_0]
+                                    Filter Operator [FIL_251] (rows=144002668 width=135)
+                                      predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null)
+                                       Please refer to the previous TableScan [TS_60]
+            <-Reducer 4 [SIMPLE_EDGE]
+              SHUFFLE [RS_125]
+                PartitionCols:_col0
+                Group By Operator [GBY_18] (rows=348477374 width=88)
+                  Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
+                <-Reducer 3 [SIMPLE_EDGE]
+                  SHUFFLE [RS_17]
+                    PartitionCols:_col0
+                    Group By Operator [GBY_16] (rows=696954748 width=88)
+                      Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7
+                      Merge Join Operator [MERGEJOIN_255] (rows=696954748 width=88)
+                        Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col2","_col7"]
+                      <-Map 23 [SIMPLE_EDGE]
+                        SHUFFLE [RS_13]
+                          PartitionCols:_col0
+                          Select Operator [SEL_8] (rows=40000000 width=1014)
+                            Output:["_col0","_col1"]
+                            Filter Operator [FIL_238] (rows=40000000 width=1014)
+                              predicate:(ca_address_sk is not null and ca_county is not null)
+                               Please refer to the previous TableScan [TS_6]
+                      <-Reducer 2 [SIMPLE_EDGE]
+                        SHUFFLE [RS_12]
+                          PartitionCols:_col1
+                          Merge Join Operator [MERGEJOIN_254] (rows=633595212 width=88)
+                            Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col1","_col2"]
+                          <-Map 12 [SIMPLE_EDGE]
+                            SHUFFLE [RS_10]
+                              PartitionCols:_col0
+                              Select Operator [SEL_5] (rows=18262 width=1119)
+                                Output:["_col0"]
+                                Filter Operator [FIL_237] (rows=18262 width=1119)
+                                  predicate:((d_qoy = 2) and (d_year = 2000) and d_date_sk is not null)
+                                   Please refer to the previous TableScan [TS_3]
+                          <-Map 1 [SIMPLE_EDGE]
+                            SHUFFLE [RS_9]
+                              PartitionCols:_col0
+                              Select Operator [SEL_2] (rows=575995635 width=88)
+                                Output:["_col0","_col1","_col2"]
+                                Filter Operator [FIL_236] (rows=575995635 width=88)
+                                  predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null)
+                                   Please refer to the previous TableScan [TS_0]
+            <-Reducer 8 [SIMPLE_EDGE]
+              SHUFFLE [RS_126]
+                PartitionCols:_col0
+                Group By Operator [GBY_38] (rows=348477374 width=88)
+                  Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
+                <-Reducer 7 [SIMPLE_EDGE]
+                  SHUFFLE [RS_37]
+                    PartitionCols:_col0
+                    Group By Operator [GBY_36] (rows=696954748 width=88)
+                      Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7
+                      Merge Join Operator [MERGEJOIN_257] (rows=696954748 width=88)
+                        Conds:RS_32._col1=RS_33._col0(Inner),Output:["_col2","_col7"]
+                      <-Map 23 [SIMPLE_EDGE]
+                        SHUFFLE [RS_33]
+                          PartitionCols:_col0
+                          Select Operator [SEL_28] (rows=40000000 width=1014)
+                            Output:["_col0","_col1"]
+                            Filter Operator [FIL_241] (rows=40000000 width=1014)
+                              predicate:(ca_address_sk is not null and ca_county is not null)
+                               Please refer to the previous TableScan [TS_6]
+                      <-Reducer 6 [SIMPLE_EDGE]
+                        SHUFFLE [RS_32]
+                          PartitionCols:_col1
+                          Merge Join Operator [MERGEJOIN_256] (rows=633595212 width=88)
+                            Conds:RS_29._col0=RS_30._col0(Inner),Output:["_col1","_col2"]
+                          <-Map 12 [SIMPLE_EDGE]
+                            SHUFFLE [RS_30]
+                              PartitionCols:_col0
+                              Select Operator [SEL_25] (rows=18262 width=1119)
+                                Output:["_col0"]
+                                Filter Operator [FIL_240] (rows=18262 width=1119)
+                                  predicate:((d_qoy = 1) and (d_year = 2000) and d_date_sk is not null)
+                                   Please refer to the previous TableScan [TS_3]
+                          <-Map 1 [SIMPLE_EDGE]
+                            SHUFFLE [RS_29]
+                              PartitionCols:_col0
+                              Select Operator [SEL_22] (rows=575995635 width=88)
+                                Output:["_col0","_col1","_col2"]
+                                Filter Operator [FIL_239] (rows=575995635 width=88)
+                                  predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null)
+                                   Please refer to the previous TableScan [TS_0]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query32.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query32.q.out b/ql/src/test/results/clientpositive/perf/query32.q.out
index c9820b3..66b8485 100644
--- a/ql/src/test/results/clientpositive/perf/query32.q.out
+++ b/ql/src/test/results/clientpositive/perf/query32.q.out
@@ -1,130 +1,152 @@
-PREHOOK: query: explain SELECT sum(cs1.cs_ext_discount_amt) as excess_discount_amount
-FROM (SELECT cs.cs_item_sk as cs_item_sk,
-                             cs.cs_ext_discount_amt as cs_ext_discount_amt
-             FROM catalog_sales cs
-             JOIN date_dim d ON (d.d_date_sk = cs.cs_sold_date_sk)
-             WHERE d.d_date between '2000-01-27' and '2000-04-27') cs1
-JOIN item i ON (i.i_item_sk = cs1.cs_item_sk)
-JOIN (SELECT cs2.cs_item_sk as cs_item_sk,
-                          1.3 * avg(cs_ext_discount_amt) as avg_cs_ext_discount_amt
-           FROM (SELECT cs.cs_item_sk as cs_item_sk,
-                                        cs.cs_ext_discount_amt as cs_ext_discount_amt
-                        FROM catalog_sales cs
-                        JOIN date_dim d ON (d.d_date_sk = cs.cs_sold_date_sk)
-                        WHERE d.d_date between '2000-01-27' and '2000-04-27') cs2
-                        GROUP BY cs2.cs_item_sk) tmp1
-ON (i.i_item_sk = tmp1.cs_item_sk)
-WHERE i.i_manufact_id = 436 and
-               cs1.cs_ext_discount_amt > tmp1.avg_cs_ext_discount_amt
+PREHOOK: query: explain
+select  sum(cs_ext_discount_amt)  as `excess discount amount` 
+from 
+   catalog_sales 
+   ,item 
+   ,date_dim
+where
+i_manufact_id = 269
+and i_item_sk = cs_item_sk 
+and d_date between '1998-03-18' and 
+        (cast('1998-03-18' as date) + 90 days)
+and d_date_sk = cs_sold_date_sk 
+and cs_ext_discount_amt  
+     > ( 
+         select 
+            1.3 * avg(cs_ext_discount_amt) 
+         from 
+            catalog_sales 
+           ,date_dim
+         where 
+              cs_item_sk = i_item_sk 
+          and d_date between '1998-03-18' and
+                             (cast('1998-03-18' as date) + 90 days)
+          and d_date_sk = cs_sold_date_sk 
+      ) 
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain SELECT sum(cs1.cs_ext_discount_amt) as excess_discount_amount
-FROM (SELECT cs.cs_item_sk as cs_item_sk,
-                             cs.cs_ext_discount_amt as cs_ext_discount_amt
-             FROM catalog_sales cs
-             JOIN date_dim d ON (d.d_date_sk = cs.cs_sold_date_sk)
-             WHERE d.d_date between '2000-01-27' and '2000-04-27') cs1
-JOIN item i ON (i.i_item_sk = cs1.cs_item_sk)
-JOIN (SELECT cs2.cs_item_sk as cs_item_sk,
-                          1.3 * avg(cs_ext_discount_amt) as avg_cs_ext_discount_amt
-           FROM (SELECT cs.cs_item_sk as cs_item_sk,
-                                        cs.cs_ext_discount_amt as cs_ext_discount_amt
-                        FROM catalog_sales cs
-                        JOIN date_dim d ON (d.d_date_sk = cs.cs_sold_date_sk)
-                        WHERE d.d_date between '2000-01-27' and '2000-04-27') cs2
-                        GROUP BY cs2.cs_item_sk) tmp1
-ON (i.i_item_sk = tmp1.cs_item_sk)
-WHERE i.i_manufact_id = 436 and
-               cs1.cs_ext_discount_amt > tmp1.avg_cs_ext_discount_amt
+POSTHOOK: query: explain
+select  sum(cs_ext_discount_amt)  as `excess discount amount` 
+from 
+   catalog_sales 
+   ,item 
+   ,date_dim
+where
+i_manufact_id = 269
+and i_item_sk = cs_item_sk 
+and d_date between '1998-03-18' and 
+        (cast('1998-03-18' as date) + 90 days)
+and d_date_sk = cs_sold_date_sk 
+and cs_ext_discount_amt  
+     > ( 
+         select 
+            1.3 * avg(cs_ext_discount_amt) 
+         from 
+            catalog_sales 
+           ,date_dim
+         where 
+              cs_item_sk = i_item_sk 
+          and d_date between '1998-03-18' and
+                             (cast('1998-03-18' as date) + 90 days)
+          and d_date_sk = cs_sold_date_sk 
+      ) 
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 8 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
-Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
+Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Map 9 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:-1
+    limit:100
     Stage-1
-      Reducer 5
-      File Output Operator [FS_37]
-        Group By Operator [GBY_35] (rows=1 width=112)
-          Output:["_col0"],aggregations:["sum(VALUE._col0)"]
-        <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
-          PARTITION_ONLY_SHUFFLE [RS_34]
-            Group By Operator [GBY_33] (rows=1 width=112)
-              Output:["_col0"],aggregations:["sum(_col5)"]
-              Select Operator [SEL_32] (rows=232311810 width=135)
-                Output:["_col5"]
-                Filter Operator [FIL_31] (rows=232311810 width=135)
-                  predicate:(_col5 > _col1)
-                  Merge Join Operator [MERGEJOIN_59] (rows=696935432 width=135)
-                    Conds:RS_27._col0=RS_28._col0(Inner),RS_28._col0=RS_29._col0(Inner),Output:["_col1","_col5"]
-                  <-Map 8 [SIMPLE_EDGE]
-                    SHUFFLE [RS_28]
-                      PartitionCols:_col0
-                      Select Operator [SEL_16] (rows=231000 width=1436)
-                        Output:["_col0"]
-                        Filter Operator [FIL_54] (rows=231000 width=1436)
-                          predicate:((i_manufact_id = 436) and i_item_sk is not null)
-                          TableScan [TS_14] (rows=462000 width=1436)
-                            default@item,i,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_manufact_id"]
-                  <-Reducer 3 [SIMPLE_EDGE]
-                    SHUFFLE [RS_27]
-                      PartitionCols:_col0
-                      Select Operator [SEL_13] (rows=158394413 width=135)
-                        Output:["_col0","_col1"]
-                        Group By Operator [GBY_12] (rows=158394413 width=135)
-                          Output:["_col0","_col1"],aggregations:["avg(VALUE._col0)"],keys:KEY._col0
-                        <-Reducer 2 [SIMPLE_EDGE]
-                          SHUFFLE [RS_11]
-                            PartitionCols:_col0
-                            Group By Operator [GBY_10] (rows=316788826 width=135)
-                              Output:["_col0","_col1"],aggregations:["avg(_col2)"],keys:_col1
-                              Merge Join Operator [MERGEJOIN_57] (rows=316788826 width=135)
-                                Conds:RS_6._col0=RS_7._col0(Inner),Output:["_col1","_col2"]
-                              <-Map 1 [SIMPLE_EDGE]
-                                SHUFFLE [RS_6]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_2] (rows=287989836 width=135)
-                                    Output:["_col0","_col1","_col2"]
-                                    Filter Operator [FIL_52] (rows=287989836 width=135)
-                                      predicate:(cs_sold_date_sk is not null and cs_item_sk is not null)
-                                      TableScan [TS_0] (rows=287989836 width=135)
-                                        default@catalog_sales,cs,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_ext_discount_amt"]
-                              <-Map 7 [SIMPLE_EDGE]
-                                SHUFFLE [RS_7]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_5] (rows=8116 width=1119)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_53] (rows=8116 width=1119)
-                                      predicate:(d_date BETWEEN '2000-01-27' AND '2000-04-27' and d_date_sk is not null)
-                                      TableScan [TS_3] (rows=73049 width=1119)
-                                        default@date_dim,d,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
-                  <-Reducer 6 [SIMPLE_EDGE]
-                    SHUFFLE [RS_29]
-                      PartitionCols:_col0
-                      Select Operator [SEL_26] (rows=316788826 width=135)
-                        Output:["_col0","_col1"]
+      Reducer 4
+      File Output Operator [FS_40]
+        Limit [LIM_39] (rows=1 width=112)
+          Number of rows:100
+          Group By Operator [GBY_37] (rows=1 width=112)
+            Output:["_col0"],aggregations:["sum(VALUE._col0)"]
+          <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
+            PARTITION_ONLY_SHUFFLE [RS_36]
+              Group By Operator [GBY_35] (rows=1 width=112)
+                Output:["_col0"],aggregations:["sum(_col2)"]
+                Select Operator [SEL_34] (rows=116155905 width=135)
+                  Output:["_col2"]
+                  Filter Operator [FIL_33] (rows=116155905 width=135)
+                    predicate:(_col2 > _col5)
+                    Merge Join Operator [MERGEJOIN_61] (rows=348467716 width=135)
+                      Conds:RS_30._col1=RS_31._col2(Inner),Output:["_col2","_col5"]
+                    <-Reducer 2 [SIMPLE_EDGE]
+                      SHUFFLE [RS_30]
+                        PartitionCols:_col1
                         Merge Join Operator [MERGEJOIN_58] (rows=316788826 width=135)
-                          Conds:RS_23._col0=RS_24._col0(Inner),Output:["_col1","_col2"]
+                          Conds:RS_27._col0=RS_28._col0(Inner),Output:["_col1","_col2"]
                         <-Map 1 [SIMPLE_EDGE]
-                          SHUFFLE [RS_23]
+                          SHUFFLE [RS_27]
                             PartitionCols:_col0
-                            Select Operator [SEL_19] (rows=287989836 width=135)
+                            Select Operator [SEL_2] (rows=287989836 width=135)
                               Output:["_col0","_col1","_col2"]
-                              Filter Operator [FIL_55] (rows=287989836 width=135)
-                                predicate:(cs_sold_date_sk is not null and cs_item_sk is not null)
-                                 Please refer to the previous TableScan [TS_0]
-                        <-Map 7 [SIMPLE_EDGE]
+                              Filter Operator [FIL_53] (rows=287989836 width=135)
+                                predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
+                                TableScan [TS_0] (rows=287989836 width=135)
+                                  default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_ext_discount_amt"]
+                        <-Map 8 [SIMPLE_EDGE]
+                          SHUFFLE [RS_28]
+                            PartitionCols:_col0
+                            Select Operator [SEL_5] (rows=8116 width=1119)
+                              Output:["_col0"]
+                              Filter Operator [FIL_54] (rows=8116 width=1119)
+                                predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1998-03-18 00:00:00.0 AND 1998-06-16 01:00:00.0 and d_date_sk is not null)
+                                TableScan [TS_3] (rows=73049 width=1119)
+                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
+                    <-Reducer 7 [SIMPLE_EDGE]
+                      SHUFFLE [RS_31]
+                        PartitionCols:_col2
+                        Merge Join Operator [MERGEJOIN_60] (rows=174233858 width=135)
+                          Conds:RS_23._col1=RS_24._col0(Inner),Output:["_col0","_col2"]
+                        <-Map 9 [SIMPLE_EDGE]
                           SHUFFLE [RS_24]
                             PartitionCols:_col0
-                            Select Operator [SEL_22] (rows=8116 width=1119)
+                            Select Operator [SEL_22] (rows=231000 width=1436)
                               Output:["_col0"]
-                              Filter Operator [FIL_56] (rows=8116 width=1119)
-                                predicate:(d_date BETWEEN '2000-01-27' AND '2000-04-27' and d_date_sk is not null)
-                                 Please refer to the previous TableScan [TS_3]
+                              Filter Operator [FIL_57] (rows=231000 width=1436)
+                                predicate:((i_manufact_id = 269) and i_item_sk is not null)
+                                TableScan [TS_20] (rows=462000 width=1436)
+                                  default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_manufact_id"]
+                        <-Reducer 6 [SIMPLE_EDGE]
+                          SHUFFLE [RS_23]
+                            PartitionCols:_col1
+                            Select Operator [SEL_19] (rows=158394413 width=135)
+                              Output:["_col0","_col1"]
+                              Group By Operator [GBY_18] (rows=158394413 width=135)
+                                Output:["_col0","_col1"],aggregations:["avg(VALUE._col0)"],keys:KEY._col0
+                              <-Reducer 5 [SIMPLE_EDGE]
+                                SHUFFLE [RS_17]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_16] (rows=316788826 width=135)
+                                    Output:["_col0","_col1"],aggregations:["avg(_col2)"],keys:_col1
+                                    Merge Join Operator [MERGEJOIN_59] (rows=316788826 width=135)
+                                      Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col1","_col2"]
+                                    <-Map 1 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_12]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_8] (rows=287989836 width=135)
+                                          Output:["_col0","_col1","_col2"]
+                                          Filter Operator [FIL_55] (rows=287989836 width=135)
+                                            predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
+                                             Please refer to the previous TableScan [TS_0]
+                                    <-Map 8 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_13]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_11] (rows=8116 width=1119)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_56] (rows=8116 width=1119)
+                                            predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1998-03-18 00:00:00.0 AND 1998-06-16 01:00:00.0 and d_date_sk is not null)
+                                             Please refer to the previous TableScan [TS_3]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query33.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query33.q.out b/ql/src/test/results/clientpositive/perf/query33.q.out
index 00c021b..c1a5fa2 100644
--- a/ql/src/test/results/clientpositive/perf/query33.q.out
+++ b/ql/src/test/results/clientpositive/perf/query33.q.out
@@ -1,4 +1,5 @@
-PREHOOK: query: explain with ss as (
+PREHOOK: query: explain
+with ss as (
  select
           i_manufact_id,sum(ss_ext_sales_price) total_sales
  from
@@ -71,7 +72,8 @@ where i_category in ('Books'))
  order by total_sales
 limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with ss as (
+POSTHOOK: query: explain
+with ss as (
  select
           i_manufact_id,sum(ss_ext_sales_price) total_sales
  from


[03/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query88.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query88.q.out b/ql/src/test/results/clientpositive/perf/query88.q.out
index f7af4ef..18d0a77 100644
--- a/ql/src/test/results/clientpositive/perf/query88.q.out
+++ b/ql/src/test/results/clientpositive/perf/query88.q.out
@@ -4,9 +4,9 @@ select  *
 from
  (select count(*) h8_30_to_9
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk   
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk 
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk   
+     and ss_hdemo_sk = household_demographics.hd_demo_sk 
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 8
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -15,9 +15,9 @@ from
      and store.s_store_name = 'ese') s1,
  (select count(*) h9_to_9_30 
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk 
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk 
      and time_dim.t_hour = 9 
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -26,9 +26,9 @@ from
      and store.s_store_name = 'ese') s2,
  (select count(*) h9_30_to_10 
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 9
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -37,9 +37,9 @@ from
      and store.s_store_name = 'ese') s3,
  (select count(*) h10_to_10_30
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 10 
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -48,9 +48,9 @@ from
      and store.s_store_name = 'ese') s4,
  (select count(*) h10_30_to_11
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 10 
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -59,9 +59,9 @@ from
      and store.s_store_name = 'ese') s5,
  (select count(*) h11_to_11_30
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk 
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk 
      and time_dim.t_hour = 11
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -70,9 +70,9 @@ from
      and store.s_store_name = 'ese') s6,
  (select count(*) h11_30_to_12
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 11
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -81,9 +81,9 @@ from
      and store.s_store_name = 'ese') s7,
  (select count(*) h12_to_12_30
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 12
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -96,9 +96,9 @@ select  *
 from
  (select count(*) h8_30_to_9
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk   
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk 
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk   
+     and ss_hdemo_sk = household_demographics.hd_demo_sk 
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 8
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -107,9 +107,9 @@ from
      and store.s_store_name = 'ese') s1,
  (select count(*) h9_to_9_30 
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk 
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk 
      and time_dim.t_hour = 9 
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -118,9 +118,9 @@ from
      and store.s_store_name = 'ese') s2,
  (select count(*) h9_30_to_10 
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 9
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -129,9 +129,9 @@ from
      and store.s_store_name = 'ese') s3,
  (select count(*) h10_to_10_30
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 10 
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -140,9 +140,9 @@ from
      and store.s_store_name = 'ese') s4,
  (select count(*) h10_30_to_11
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 10 
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -151,9 +151,9 @@ from
      and store.s_store_name = 'ese') s5,
  (select count(*) h11_to_11_30
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk 
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk 
      and time_dim.t_hour = 11
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -162,9 +162,9 @@ from
      and store.s_store_name = 'ese') s6,
  (select count(*) h11_30_to_12
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 11
      and time_dim.t_minute >= 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
@@ -173,9 +173,9 @@ from
      and store.s_store_name = 'ese') s7,
  (select count(*) h12_to_12_30
  from store_sales, household_demographics , time_dim, store
- where store_sales.ss_sold_time_sk = time_dim.t_time_sk
-     and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-     and store_sales.ss_store_sk = store.s_store_sk
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
      and time_dim.t_hour = 12
      and time_dim.t_minute < 30
      and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query89.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query89.q.out b/ql/src/test/results/clientpositive/perf/query89.q.out
index 27467bd..1f13f9e 100644
--- a/ql/src/test/results/clientpositive/perf/query89.q.out
+++ b/ql/src/test/results/clientpositive/perf/query89.q.out
@@ -9,9 +9,9 @@ select i_category, i_class, i_brand,
          (partition by i_category, i_brand, s_store_name, s_company_name)
          avg_monthly_sales
 from item, store_sales, date_dim, store
-where store_sales.ss_item_sk = item.i_item_sk and
-      store_sales.ss_sold_date_sk = date_dim.d_date_sk and
-      store_sales.ss_store_sk = store.s_store_sk and
+where ss_item_sk = i_item_sk and
+      ss_sold_date_sk = d_date_sk and
+      ss_store_sk = s_store_sk and
       d_year in (2000) and
         ((i_category in ('Home','Books','Electronics') and
           i_class in ('wallpaper','parenting','musical')
@@ -36,9 +36,9 @@ select i_category, i_class, i_brand,
          (partition by i_category, i_brand, s_store_name, s_company_name)
          avg_monthly_sales
 from item, store_sales, date_dim, store
-where store_sales.ss_item_sk = item.i_item_sk and
-      store_sales.ss_sold_date_sk = date_dim.d_date_sk and
-      store_sales.ss_store_sk = store.s_store_sk and
+where ss_item_sk = i_item_sk and
+      ss_sold_date_sk = d_date_sk and
+      ss_store_sk = s_store_sk and
       d_year in (2000) and
         ((i_category in ('Home','Books','Electronics') and
           i_class in ('wallpaper','parenting','musical')

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query9.q.out b/ql/src/test/results/clientpositive/perf/query9.q.out
index 4dded91..425163e 100644
--- a/ql/src/test/results/clientpositive/perf/query9.q.out
+++ b/ql/src/test/results/clientpositive/perf/query9.q.out
@@ -13,12 +13,13 @@ Warning: Shuffle Join MERGEJOIN[182][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_
 Warning: Shuffle Join MERGEJOIN[183][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5, $hdt$_6, $hdt$_7, $hdt$_8, $hdt$_9, $hdt$_10, $hdt$_11, $hdt$_12, $hdt$_13]] in Stage 'Reducer 14' is a cross product
 Warning: Shuffle Join MERGEJOIN[184][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5, $hdt$_6, $hdt$_7, $hdt$_8, $hdt$_9, $hdt$_10, $hdt$_11, $hdt$_12, $hdt$_13, $hdt$_14]] in Stage 'Reducer 15' is a cross product
 Warning: Shuffle Join MERGEJOIN[185][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5, $hdt$_6, $hdt$_7, $hdt$_8, $hdt$_9, $hdt$_10, $hdt$_11, $hdt$_12, $hdt$_13, $hdt$_14, $hdt$_15]] in Stage 'Reducer 16' is a cross product
-PREHOOK: query: explain select case when (select count(*)
-                  from store_sales
+PREHOOK: query: explain
+select case when (select count(*) 
+                  from store_sales 
                   where ss_quantity between 1 and 20) > 409437
-            then (select avg(ss_ext_list_price)
-                  from store_sales
-                  where ss_quantity between 1 and 20)
+            then (select avg(ss_ext_list_price) 
+                  from store_sales 
+                  where ss_quantity between 1 and 20) 
             else (select avg(ss_net_paid_inc_tax)
                   from store_sales
                   where ss_quantity between 1 and 20) end bucket1 ,
@@ -27,7 +28,7 @@ PREHOOK: query: explain select case when (select count(*)
                   where ss_quantity between 21 and 40) > 4595804
             then (select avg(ss_ext_list_price)
                   from store_sales
-                  where ss_quantity between 21 and 40)
+                  where ss_quantity between 21 and 40) 
             else (select avg(ss_net_paid_inc_tax)
                   from store_sales
                   where ss_quantity between 21 and 40) end bucket2,
@@ -61,12 +62,13 @@ PREHOOK: query: explain select case when (select count(*)
 from reason
 where r_reason_sk = 1
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select case when (select count(*)
-                  from store_sales
+POSTHOOK: query: explain
+select case when (select count(*) 
+                  from store_sales 
                   where ss_quantity between 1 and 20) > 409437
-            then (select avg(ss_ext_list_price)
-                  from store_sales
-                  where ss_quantity between 1 and 20)
+            then (select avg(ss_ext_list_price) 
+                  from store_sales 
+                  where ss_quantity between 1 and 20) 
             else (select avg(ss_net_paid_inc_tax)
                   from store_sales
                   where ss_quantity between 1 and 20) end bucket1 ,
@@ -75,7 +77,7 @@ POSTHOOK: query: explain select case when (select count(*)
                   where ss_quantity between 21 and 40) > 4595804
             then (select avg(ss_ext_list_price)
                   from store_sales
-                  where ss_quantity between 21 and 40)
+                  where ss_quantity between 21 and 40) 
             else (select avg(ss_net_paid_inc_tax)
                   from store_sales
                   where ss_quantity between 21 and 40) end bucket2,

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query90.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query90.q.out b/ql/src/test/results/clientpositive/perf/query90.q.out
index aae0aec..b3468ec 100644
--- a/ql/src/test/results/clientpositive/perf/query90.q.out
+++ b/ql/src/test/results/clientpositive/perf/query90.q.out
@@ -1,7 +1,45 @@
 Warning: Shuffle Join MERGEJOIN[92][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
-PREHOOK: query: explain select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio from ( select count(*) amc from web_sales, household_demographics , time_dim, web_page where ws_sold_time_sk = time_dim.t_time_sk and ws_ship_hdemo_sk = household_demographics.hd_demo_sk and ws_web_page_sk = web_page.wp_web_page_sk and time_dim.t_hour between 6 and 6+1 and household_demographics.hd_dep_count = 8 and web_page.wp_char_count between 5000 and 5200) at, ( select count(*) pmc from web_sales, household_demographics , time_dim, web_page where ws_sold_time_sk = time_dim.t_time_sk and ws_ship_hdemo_sk = household_demographics.hd_demo_sk and ws_web_page_sk = web_page.wp_web_page_sk and time_dim.t_hour between 14 and 14+1 and household_demographics.hd_dep_count = 8 and web_page.wp_char_count between 5000 and 5200) pt order by am_pm_ratio limit 100
+PREHOOK: query: explain
+select  cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio
+ from ( select count(*) amc
+       from web_sales, household_demographics , time_dim, web_page
+       where ws_sold_time_sk = time_dim.t_time_sk
+         and ws_ship_hdemo_sk = household_demographics.hd_demo_sk
+         and ws_web_page_sk = web_page.wp_web_page_sk
+         and time_dim.t_hour between 6 and 6+1
+         and household_demographics.hd_dep_count = 8
+         and web_page.wp_char_count between 5000 and 5200) at,
+      ( select count(*) pmc
+       from web_sales, household_demographics , time_dim, web_page
+       where ws_sold_time_sk = time_dim.t_time_sk
+         and ws_ship_hdemo_sk = household_demographics.hd_demo_sk
+         and ws_web_page_sk = web_page.wp_web_page_sk
+         and time_dim.t_hour between 14 and 14+1
+         and household_demographics.hd_dep_count = 8
+         and web_page.wp_char_count between 5000 and 5200) pt
+ order by am_pm_ratio
+ limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio from ( select count(*) amc from web_sales, household_demographics , time_dim, web_page where ws_sold_time_sk = time_dim.t_time_sk and ws_ship_hdemo_sk = household_demographics.hd_demo_sk and ws_web_page_sk = web_page.wp_web_page_sk and time_dim.t_hour between 6 and 6+1 and household_demographics.hd_dep_count = 8 and web_page.wp_char_count between 5000 and 5200) at, ( select count(*) pmc from web_sales, household_demographics , time_dim, web_page where ws_sold_time_sk = time_dim.t_time_sk and ws_ship_hdemo_sk = household_demographics.hd_demo_sk and ws_web_page_sk = web_page.wp_web_page_sk and time_dim.t_hour between 14 and 14+1 and household_demographics.hd_dep_count = 8 and web_page.wp_char_count between 5000 and 5200) pt order by am_pm_ratio limit 100
+POSTHOOK: query: explain
+select  cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio
+ from ( select count(*) amc
+       from web_sales, household_demographics , time_dim, web_page
+       where ws_sold_time_sk = time_dim.t_time_sk
+         and ws_ship_hdemo_sk = household_demographics.hd_demo_sk
+         and ws_web_page_sk = web_page.wp_web_page_sk
+         and time_dim.t_hour between 6 and 6+1
+         and household_demographics.hd_dep_count = 8
+         and web_page.wp_char_count between 5000 and 5200) at,
+      ( select count(*) pmc
+       from web_sales, household_demographics , time_dim, web_page
+       where ws_sold_time_sk = time_dim.t_time_sk
+         and ws_ship_hdemo_sk = household_demographics.hd_demo_sk
+         and ws_web_page_sk = web_page.wp_web_page_sk
+         and time_dim.t_hour between 14 and 14+1
+         and household_demographics.hd_dep_count = 8
+         and web_page.wp_char_count between 5000 and 5200) pt
+ order by am_pm_ratio
+ limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query91.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query91.q.out b/ql/src/test/results/clientpositive/perf/query91.q.out
index 0ee7e43..ef04f4a 100644
--- a/ql/src/test/results/clientpositive/perf/query91.q.out
+++ b/ql/src/test/results/clientpositive/perf/query91.q.out
@@ -1,6 +1,62 @@
-PREHOOK: query: explain select cc_call_center_id Call_Center, cc_name Call_Center_Name, cc_manager Manager, sum(cr_net_loss) Returns_Loss from call_center, catalog_returns, date_dim, customer, customer_address, customer_demographics, household_demographics where catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk and catalog_returns.cr_returned_date_sk = date_dim.d_date_sk and catalog_returns.cr_returning_customer_sk= customer.c_customer_sk and customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk and household_demographics.hd_demo_sk = customer.c_current_hdemo_sk and customer_address.ca_address_sk = customer.c_current_addr_sk and d_year = 1999 and d_moy = 11 and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) and hd_buy_potential like '0-500%' and ca_gmt_offset = -7 group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status order by Returns_Loss d
 esc
+PREHOOK: query: explain
+select  
+        cc_call_center_id Call_Center,
+        cc_name Call_Center_Name,
+        cc_manager Manager,
+        sum(cr_net_loss) Returns_Loss
+from
+        call_center,
+        catalog_returns,
+        date_dim,
+        customer,
+        customer_address,
+        customer_demographics,
+        household_demographics
+where
+        cr_call_center_sk       = cc_call_center_sk
+and     cr_returned_date_sk     = d_date_sk
+and     cr_returning_customer_sk= c_customer_sk
+and     cd_demo_sk              = c_current_cdemo_sk
+and     hd_demo_sk              = c_current_hdemo_sk
+and     ca_address_sk           = c_current_addr_sk
+and     d_year                  = 1999 
+and     d_moy                   = 11
+and     ( (cd_marital_status       = 'M' and cd_education_status     = 'Unknown')
+        or(cd_marital_status       = 'W' and cd_education_status     = 'Advanced Degree'))
+and     hd_buy_potential like '0-500%'
+and     ca_gmt_offset           = -7
+group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status
+order by sum(cr_net_loss) desc
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select cc_call_center_id Call_Center, cc_name Call_Center_Name, cc_manager Manager, sum(cr_net_loss) Returns_Loss from call_center, catalog_returns, date_dim, customer, customer_address, customer_demographics, household_demographics where catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk and catalog_returns.cr_returned_date_sk = date_dim.d_date_sk and catalog_returns.cr_returning_customer_sk= customer.c_customer_sk and customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk and household_demographics.hd_demo_sk = customer.c_current_hdemo_sk and customer_address.ca_address_sk = customer.c_current_addr_sk and d_year = 1999 and d_moy = 11 and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) and hd_buy_potential like '0-500%' and ca_gmt_offset = -7 group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status order by Returns_Loss 
 desc
+POSTHOOK: query: explain
+select  
+        cc_call_center_id Call_Center,
+        cc_name Call_Center_Name,
+        cc_manager Manager,
+        sum(cr_net_loss) Returns_Loss
+from
+        call_center,
+        catalog_returns,
+        date_dim,
+        customer,
+        customer_address,
+        customer_demographics,
+        household_demographics
+where
+        cr_call_center_sk       = cc_call_center_sk
+and     cr_returned_date_sk     = d_date_sk
+and     cr_returning_customer_sk= c_customer_sk
+and     cd_demo_sk              = c_current_cdemo_sk
+and     hd_demo_sk              = c_current_hdemo_sk
+and     ca_address_sk           = c_current_addr_sk
+and     d_year                  = 1999 
+and     d_moy                   = 11
+and     ( (cd_marital_status       = 'M' and cd_education_status     = 'Unknown')
+        or(cd_marital_status       = 'W' and cd_education_status     = 'Advanced Degree'))
+and     hd_buy_potential like '0-500%'
+and     ca_gmt_offset           = -7
+group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status
+order by sum(cr_net_loss) desc
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -25,7 +81,7 @@ Stage-0
         <-Reducer 5 [SIMPLE_EDGE]
           SHUFFLE [RS_45]
             Select Operator [SEL_44] (rows=58564004 width=860)
-              Output:["_col0","_col1","_col2","_col3"]
+              Output:["_col0","_col1","_col2","_col6"]
               Group By Operator [GBY_43] (rows=58564004 width=860)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
               <-Reducer 4 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query92.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query92.q.out b/ql/src/test/results/clientpositive/perf/query92.q.out
index d3abc23..ec4fbb9 100644
--- a/ql/src/test/results/clientpositive/perf/query92.q.out
+++ b/ql/src/test/results/clientpositive/perf/query92.q.out
@@ -1,90 +1,163 @@
-PREHOOK: query: explain SELECT sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) as store_only, sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) as catalog_only, sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) as store_and_catalog FROM (SELECT ss.ss_customer_sk as customer_sk, ss.ss_item_sk as item_sk FROM store_sales ss JOIN date_dim d1 ON (ss.ss_sold_date_sk = d1.d_date_sk) WHERE d1.d_month_seq >= 1206 and d1.d_month_seq <= 1217 GROUP BY ss.ss_customer_sk, ss.ss_item_sk) ssci FULL OUTER JOIN (SELECT cs.cs_bill_customer_sk as customer_sk, cs.cs_item_sk as item_sk FROM catalog_sales cs JOIN date_dim d2 ON (cs.cs_sold_date_sk = d2.d_date_sk) WHERE d2.d_month_seq >= 1206 and d2.d_month_seq <= 1217 GROUP BY cs.cs_bill_customer_sk, cs.cs_item_sk) csci ON (ssci.customer_sk=csci.customer_sk and ssci.item_sk = csci.item_sk)
+PREHOOK: query: explain
+select  
+   sum(ws_ext_discount_amt)  as `Excess Discount Amount` 
+from 
+    web_sales 
+   ,item 
+   ,date_dim
+where
+i_manufact_id = 269
+and i_item_sk = ws_item_sk 
+and d_date between '1998-03-18' and 
+        (cast('1998-03-18' as date) + 90 days)
+and d_date_sk = ws_sold_date_sk 
+and ws_ext_discount_amt  
+     > ( 
+         SELECT 
+            1.3 * avg(ws_ext_discount_amt) 
+         FROM 
+            web_sales 
+           ,date_dim
+         WHERE 
+              ws_item_sk = i_item_sk 
+          and d_date between '1998-03-18' and
+                             (cast('1998-03-18' as date) + 90 days)
+          and d_date_sk = ws_sold_date_sk 
+      ) 
+order by sum(ws_ext_discount_amt)
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain SELECT sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) as store_only, sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) as catalog_only, sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) as store_and_catalog FROM (SELECT ss.ss_customer_sk as customer_sk, ss.ss_item_sk as item_sk FROM store_sales ss JOIN date_dim d1 ON (ss.ss_sold_date_sk = d1.d_date_sk) WHERE d1.d_month_seq >= 1206 and d1.d_month_seq <= 1217 GROUP BY ss.ss_customer_sk, ss.ss_item_sk) ssci FULL OUTER JOIN (SELECT cs.cs_bill_customer_sk as customer_sk, cs.cs_item_sk as item_sk FROM catalog_sales cs JOIN date_dim d2 ON (cs.cs_sold_date_sk = d2.d_date_sk) WHERE d2.d_month_seq >= 1206 and d2.d_month_seq <= 1217 GROUP BY cs.cs_bill_customer_sk, cs.cs_item_sk) csci ON (ssci.customer_sk=csci.customer_sk and ssci.item_sk = csci.item_sk)
+POSTHOOK: query: explain
+select  
+   sum(ws_ext_discount_amt)  as `Excess Discount Amount` 
+from 
+    web_sales 
+   ,item 
+   ,date_dim
+where
+i_manufact_id = 269
+and i_item_sk = ws_item_sk 
+and d_date between '1998-03-18' and 
+        (cast('1998-03-18' as date) + 90 days)
+and d_date_sk = ws_sold_date_sk 
+and ws_ext_discount_amt  
+     > ( 
+         SELECT 
+            1.3 * avg(ws_ext_discount_amt) 
+         FROM 
+            web_sales 
+           ,date_dim
+         WHERE 
+              ws_item_sk = i_item_sk 
+          and d_date between '1998-03-18' and
+                             (cast('1998-03-18' as date) + 90 days)
+          and d_date_sk = ws_sold_date_sk 
+      ) 
+order by sum(ws_ext_discount_amt)
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
-Reducer 7 <- Map 6 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
-Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+Reducer 8 <- Map 10 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
       Reducer 5
-      File Output Operator [FS_37]
-        Group By Operator [GBY_35] (rows=1 width=24)
-          Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
-        <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
-          PARTITION_ONLY_SHUFFLE [RS_34]
-            Group By Operator [GBY_33] (rows=1 width=24)
-              Output:["_col0","_col1","_col2"],aggregations:["sum(_col0)","sum(_col1)","sum(_col2)"]
-              Select Operator [SEL_31] (rows=348477374 width=88)
-                Output:["_col0","_col1","_col2"]
-                Merge Join Operator [MERGEJOIN_48] (rows=348477374 width=88)
-                  Conds:RS_28._col0, _col1=RS_29._col0, _col1(Outer),Output:["_col0","_col2"]
-                <-Reducer 3 [SIMPLE_EDGE]
-                  SHUFFLE [RS_28]
-                    PartitionCols:_col0, _col1
-                    Group By Operator [GBY_12] (rows=316797606 width=88)
-                      Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                    <-Reducer 2 [SIMPLE_EDGE]
-                      SHUFFLE [RS_11]
-                        PartitionCols:_col0, _col1
-                        Group By Operator [GBY_10] (rows=633595212 width=88)
-                          Output:["_col0","_col1"],keys:_col2, _col1
-                          Merge Join Operator [MERGEJOIN_46] (rows=633595212 width=88)
-                            Conds:RS_6._col0=RS_7._col0(Inner),Output:["_col1","_col2"]
-                          <-Map 6 [SIMPLE_EDGE]
-                            SHUFFLE [RS_7]
-                              PartitionCols:_col0
-                              Select Operator [SEL_5] (rows=8116 width=1119)
-                                Output:["_col0"]
-                                Filter Operator [FIL_43] (rows=8116 width=1119)
-                                  predicate:((d_month_seq >= 1206) and (d_month_seq <= 1217) and d_date_sk is not null)
-                                  TableScan [TS_3] (rows=73049 width=1119)
-                                    default@date_dim,d1,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_month_seq"]
-                          <-Map 1 [SIMPLE_EDGE]
-                            SHUFFLE [RS_6]
-                              PartitionCols:_col0
-                              Select Operator [SEL_2] (rows=575995635 width=88)
-                                Output:["_col0","_col1","_col2"]
-                                Filter Operator [FIL_42] (rows=575995635 width=88)
-                                  predicate:ss_sold_date_sk is not null
-                                  TableScan [TS_0] (rows=575995635 width=88)
-                                    default@store_sales,ss,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk"]
-                <-Reducer 8 [SIMPLE_EDGE]
-                  SHUFFLE [RS_29]
-                    PartitionCols:_col0, _col1
-                    Group By Operator [GBY_26] (rows=158394413 width=135)
-                      Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                    <-Reducer 7 [SIMPLE_EDGE]
-                      SHUFFLE [RS_25]
-                        PartitionCols:_col0, _col1
-                        Group By Operator [GBY_24] (rows=316788826 width=135)
-                          Output:["_col0","_col1"],keys:_col1, _col2
-                          Merge Join Operator [MERGEJOIN_47] (rows=316788826 width=135)
-                            Conds:RS_20._col0=RS_21._col0(Inner),Output:["_col1","_col2"]
-                          <-Map 6 [SIMPLE_EDGE]
-                            SHUFFLE [RS_21]
-                              PartitionCols:_col0
-                              Select Operator [SEL_19] (rows=8116 width=1119)
-                                Output:["_col0"]
-                                Filter Operator [FIL_45] (rows=8116 width=1119)
-                                  predicate:((d_month_seq >= 1206) and (d_month_seq <= 1217) and d_date_sk is not null)
-                                   Please refer to the previous TableScan [TS_3]
-                          <-Map 9 [SIMPLE_EDGE]
-                            SHUFFLE [RS_20]
-                              PartitionCols:_col0
-                              Select Operator [SEL_16] (rows=287989836 width=135)
-                                Output:["_col0","_col1","_col2"]
-                                Filter Operator [FIL_44] (rows=287989836 width=135)
-                                  predicate:cs_sold_date_sk is not null
-                                  TableScan [TS_14] (rows=287989836 width=135)
-                                    default@catalog_sales,cs,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_bill_customer_sk","cs_item_sk"]
+      File Output Operator [FS_43]
+        Limit [LIM_41] (rows=1 width=112)
+          Number of rows:100
+          Select Operator [SEL_40] (rows=1 width=112)
+            Output:["_col0"]
+          <-Reducer 4 [SIMPLE_EDGE]
+            SHUFFLE [RS_39]
+              Select Operator [SEL_38] (rows=1 width=112)
+                Output:["_col1"]
+                Group By Operator [GBY_37] (rows=1 width=112)
+                  Output:["_col0"],aggregations:["sum(VALUE._col0)"]
+                <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_36]
+                    Group By Operator [GBY_35] (rows=1 width=112)
+                      Output:["_col0"],aggregations:["sum(_col2)"]
+                      Select Operator [SEL_34] (rows=58081078 width=135)
+                        Output:["_col2"]
+                        Filter Operator [FIL_33] (rows=58081078 width=135)
+                          predicate:(_col2 > _col5)
+                          Merge Join Operator [MERGEJOIN_64] (rows=174243235 width=135)
+                            Conds:RS_30._col1=RS_31._col2(Inner),Output:["_col2","_col5"]
+                          <-Reducer 2 [SIMPLE_EDGE]
+                            SHUFFLE [RS_30]
+                              PartitionCols:_col1
+                              Merge Join Operator [MERGEJOIN_61] (rows=158402938 width=135)
+                                Conds:RS_27._col0=RS_28._col0(Inner),Output:["_col1","_col2"]
+                              <-Map 1 [SIMPLE_EDGE]
+                                SHUFFLE [RS_27]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_2] (rows=144002668 width=135)
+                                    Output:["_col0","_col1","_col2"]
+                                    Filter Operator [FIL_56] (rows=144002668 width=135)
+                                      predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
+                                      TableScan [TS_0] (rows=144002668 width=135)
+                                        default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_ext_discount_amt"]
+                              <-Map 9 [SIMPLE_EDGE]
+                                SHUFFLE [RS_28]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_5] (rows=8116 width=1119)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_57] (rows=8116 width=1119)
+                                      predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1998-03-18 00:00:00.0 AND 1998-06-16 01:00:00.0 and d_date_sk is not null)
+                                      TableScan [TS_3] (rows=73049 width=1119)
+                                        default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
+                          <-Reducer 8 [SIMPLE_EDGE]
+                            SHUFFLE [RS_31]
+                              PartitionCols:_col2
+                              Merge Join Operator [MERGEJOIN_63] (rows=87121617 width=135)
+                                Conds:RS_23._col1=RS_24._col0(Inner),Output:["_col0","_col2"]
+                              <-Map 10 [SIMPLE_EDGE]
+                                SHUFFLE [RS_24]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_22] (rows=231000 width=1436)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_60] (rows=231000 width=1436)
+                                      predicate:((i_manufact_id = 269) and i_item_sk is not null)
+                                      TableScan [TS_20] (rows=462000 width=1436)
+                                        default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_manufact_id"]
+                              <-Reducer 7 [SIMPLE_EDGE]
+                                SHUFFLE [RS_23]
+                                  PartitionCols:_col1
+                                  Select Operator [SEL_19] (rows=79201469 width=135)
+                                    Output:["_col0","_col1"]
+                                    Group By Operator [GBY_18] (rows=79201469 width=135)
+                                      Output:["_col0","_col1"],aggregations:["avg(VALUE._col0)"],keys:KEY._col0
+                                    <-Reducer 6 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_17]
+                                        PartitionCols:_col0
+                                        Group By Operator [GBY_16] (rows=158402938 width=135)
+                                          Output:["_col0","_col1"],aggregations:["avg(_col2)"],keys:_col1
+                                          Merge Join Operator [MERGEJOIN_62] (rows=158402938 width=135)
+                                            Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col1","_col2"]
+                                          <-Map 1 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_12]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_8] (rows=144002668 width=135)
+                                                Output:["_col0","_col1","_col2"]
+                                                Filter Operator [FIL_58] (rows=144002668 width=135)
+                                                  predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
+                                                   Please refer to the previous TableScan [TS_0]
+                                          <-Map 9 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_13]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_11] (rows=8116 width=1119)
+                                                Output:["_col0"]
+                                                Filter Operator [FIL_59] (rows=8116 width=1119)
+                                                  predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1998-03-18 00:00:00.0 AND 1998-06-16 01:00:00.0 and d_date_sk is not null)
+                                                   Please refer to the previous TableScan [TS_3]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query93.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query93.q.out b/ql/src/test/results/clientpositive/perf/query93.q.out
index f28ba41..fc58396 100644
--- a/ql/src/test/results/clientpositive/perf/query93.q.out
+++ b/ql/src/test/results/clientpositive/perf/query93.q.out
@@ -1,6 +1,36 @@
-PREHOOK: query: explain select ss_customer_sk ,sum(act_sales) sumsales from (select ss_item_sk ,ss_ticket_number ,ss_customer_sk ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price else (ss_quantity*ss_sales_price) end act_sales from store_sales left outer join store_returns on (store_returns.sr_item_sk = store_sales.ss_item_sk and store_returns.sr_ticket_number = store_sales.ss_ticket_number) ,reason where store_returns.sr_reason_sk = reason.r_reason_sk and r_reason_desc = 'Did not like the warranty') t group by ss_customer_sk order by sumsales, ss_customer_sk limit 100
+PREHOOK: query: explain
+select  ss_customer_sk
+            ,sum(act_sales) sumsales
+      from (select ss_item_sk
+                  ,ss_ticket_number
+                  ,ss_customer_sk
+                  ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price
+                                                            else (ss_quantity*ss_sales_price) end act_sales
+            from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk
+                                                               and sr_ticket_number = ss_ticket_number)
+                ,reason
+            where sr_reason_sk = r_reason_sk
+              and r_reason_desc = 'Did not like the warranty') t
+      group by ss_customer_sk
+      order by sumsales, ss_customer_sk
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select ss_customer_sk ,sum(act_sales) sumsales from (select ss_item_sk ,ss_ticket_number ,ss_customer_sk ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price else (ss_quantity*ss_sales_price) end act_sales from store_sales left outer join store_returns on (store_returns.sr_item_sk = store_sales.ss_item_sk and store_returns.sr_ticket_number = store_sales.ss_ticket_number) ,reason where store_returns.sr_reason_sk = reason.r_reason_sk and r_reason_desc = 'Did not like the warranty') t group by ss_customer_sk order by sumsales, ss_customer_sk limit 100
+POSTHOOK: query: explain
+select  ss_customer_sk
+            ,sum(act_sales) sumsales
+      from (select ss_item_sk
+                  ,ss_ticket_number
+                  ,ss_customer_sk
+                  ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price
+                                                            else (ss_quantity*ss_sales_price) end act_sales
+            from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk
+                                                               and sr_ticket_number = ss_ticket_number)
+                ,reason
+            where sr_reason_sk = r_reason_sk
+              and r_reason_desc = 'Did not like the warranty') t
+      group by ss_customer_sk
+      order by sumsales, ss_customer_sk
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query94.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query94.q.out b/ql/src/test/results/clientpositive/perf/query94.q.out
index 836b16b..6e24345 100644
--- a/ql/src/test/results/clientpositive/perf/query94.q.out
+++ b/ql/src/test/results/clientpositive/perf/query94.q.out
@@ -1,130 +1,231 @@
-PREHOOK: query: explain SELECT count(distinct ws_order_number) as order_count, sum(ws_ext_ship_cost) as total_shipping_cost, sum(ws_net_profit) as total_net_profit FROM web_sales ws1 JOIN customer_address ca ON (ws1.ws_ship_addr_sk = ca.ca_address_sk) JOIN web_site s ON (ws1.ws_web_site_sk = s.web_site_sk) JOIN date_dim d ON (ws1.ws_ship_date_sk = d.d_date_sk) LEFT SEMI JOIN (SELECT ws2.ws_order_number as ws_order_number FROM web_sales ws2 JOIN web_sales ws3 ON (ws2.ws_order_number = ws3.ws_order_number) WHERE ws2.ws_warehouse_sk <> ws3.ws_warehouse_sk ) ws_wh1 ON (ws1.ws_order_number = ws_wh1.ws_order_number) LEFT OUTER JOIN web_returns wr1 ON (ws1.ws_order_number = wr1.wr_order_number) WHERE d.d_date between '1999-05-01' and '1999-07-01' and ca.ca_state = 'TX' and s.web_company_name = 'pri' and wr1.wr_order_number is null limit 100
+Warning: Shuffle Join MERGEJOIN[107][tables = [$hdt$_2, $hdt$_3, $hdt$_1, $hdt$_4]] in Stage 'Reducer 17' is a cross product
+PREHOOK: query: explain
+select  
+   count(distinct ws_order_number) as `order count`
+  ,sum(ws_ext_ship_cost) as `total shipping cost`
+  ,sum(ws_net_profit) as `total net profit`
+from
+   web_sales ws1
+  ,date_dim
+  ,customer_address
+  ,web_site
+where
+    d_date between '1999-5-01' and 
+           (cast('1999-5-01' as date) + 60 days)
+and ws1.ws_ship_date_sk = d_date_sk
+and ws1.ws_ship_addr_sk = ca_address_sk
+and ca_state = 'TX'
+and ws1.ws_web_site_sk = web_site_sk
+and web_company_name = 'pri'
+and exists (select *
+            from web_sales ws2
+            where ws1.ws_order_number = ws2.ws_order_number
+              and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk)
+and not exists(select *
+               from web_returns wr1
+               where ws1.ws_order_number = wr1.wr_order_number)
+order by count(distinct ws_order_number)
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain SELECT count(distinct ws_order_number) as order_count, sum(ws_ext_ship_cost) as total_shipping_cost, sum(ws_net_profit) as total_net_profit FROM web_sales ws1 JOIN customer_address ca ON (ws1.ws_ship_addr_sk = ca.ca_address_sk) JOIN web_site s ON (ws1.ws_web_site_sk = s.web_site_sk) JOIN date_dim d ON (ws1.ws_ship_date_sk = d.d_date_sk) LEFT SEMI JOIN (SELECT ws2.ws_order_number as ws_order_number FROM web_sales ws2 JOIN web_sales ws3 ON (ws2.ws_order_number = ws3.ws_order_number) WHERE ws2.ws_warehouse_sk <> ws3.ws_warehouse_sk ) ws_wh1 ON (ws1.ws_order_number = ws_wh1.ws_order_number) LEFT OUTER JOIN web_returns wr1 ON (ws1.ws_order_number = wr1.wr_order_number) WHERE d.d_date between '1999-05-01' and '1999-07-01' and ca.ca_state = 'TX' and s.web_company_name = 'pri' and wr1.wr_order_number is null limit 100
+POSTHOOK: query: explain
+select  
+   count(distinct ws_order_number) as `order count`
+  ,sum(ws_ext_ship_cost) as `total shipping cost`
+  ,sum(ws_net_profit) as `total net profit`
+from
+   web_sales ws1
+  ,date_dim
+  ,customer_address
+  ,web_site
+where
+    d_date between '1999-5-01' and 
+           (cast('1999-5-01' as date) + 60 days)
+and ws1.ws_ship_date_sk = d_date_sk
+and ws1.ws_ship_addr_sk = ca_address_sk
+and ca_state = 'TX'
+and ws1.ws_web_site_sk = web_site_sk
+and web_company_name = 'pri'
+and exists (select *
+            from web_sales ws2
+            where ws1.ws_order_number = ws2.ws_order_number
+              and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk)
+and not exists(select *
+               from web_returns wr1
+               where ws1.ws_order_number = wr1.wr_order_number)
+order by count(distinct ws_order_number)
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 3 <- Map 11 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 13 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-Reducer 6 <- Map 14 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+Reducer 13 <- Map 12 (SIMPLE_EDGE)
+Reducer 15 <- Map 14 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE)
+Reducer 16 <- Reducer 15 (SIMPLE_EDGE)
+Reducer 17 <- Map 14 (CUSTOM_SIMPLE_EDGE), Map 19 (CUSTOM_SIMPLE_EDGE), Map 20 (CUSTOM_SIMPLE_EDGE), Map 21 (CUSTOM_SIMPLE_EDGE)
+Reducer 18 <- Reducer 17 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 3 <- Map 10 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 11 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 13 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 16 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
 Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
-Reducer 9 <- Map 10 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
-      Reducer 7
-      File Output Operator [FS_51]
-        Limit [LIM_50] (rows=1 width=344)
+      Reducer 8
+      File Output Operator [FS_74]
+        Limit [LIM_72] (rows=1 width=344)
           Number of rows:100
-          Group By Operator [GBY_48] (rows=1 width=344)
-            Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT KEY._col0:0._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
-          <-Reducer 6 [SIMPLE_EDGE]
-            SHUFFLE [RS_47]
-              Group By Operator [GBY_46] (rows=127554770 width=135)
-                Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT _col3)","sum(_col4)","sum(_col5)"],keys:_col3
-                Select Operator [SEL_45] (rows=127554770 width=135)
-                  Output:["_col3","_col4","_col5"]
-                  Filter Operator [FIL_44] (rows=127554770 width=135)
-                    predicate:_col12 is null
-                    Merge Join Operator [MERGEJOIN_85] (rows=255109540 width=135)
-                      Conds:RS_40._col3=RS_41._col0(Left Outer),Output:["_col3","_col4","_col5","_col12"]
-                    <-Map 14 [SIMPLE_EDGE]
-                      SHUFFLE [RS_41]
-                        PartitionCols:_col0
-                        Select Operator [SEL_25] (rows=14398467 width=92)
-                          Output:["_col0"]
-                          Filter Operator [FIL_79] (rows=14398467 width=92)
-                            predicate:wr_order_number is not null
-                            TableScan [TS_23] (rows=14398467 width=92)
-                              default@web_returns,wr1,Tbl:COMPLETE,Col:NONE,Output:["wr_order_number"]
-                    <-Reducer 5 [SIMPLE_EDGE]
-                      SHUFFLE [RS_40]
-                        PartitionCols:_col3
-                        Merge Join Operator [MERGEJOIN_84] (rows=231917759 width=135)
-                          Conds:RS_37._col2=RS_38._col0(Inner),Output:["_col3","_col4","_col5"]
-                        <-Map 13 [SIMPLE_EDGE]
-                          SHUFFLE [RS_38]
-                            PartitionCols:_col0
-                            Select Operator [SEL_22] (rows=42 width=1850)
-                              Output:["_col0"]
-                              Filter Operator [FIL_78] (rows=42 width=1850)
-                                predicate:((web_company_name = 'pri') and web_site_sk is not null)
-                                TableScan [TS_20] (rows=84 width=1850)
-                                  default@web_site,s,Tbl:COMPLETE,Col:NONE,Output:["web_site_sk","web_company_name"]
-                        <-Reducer 4 [SIMPLE_EDGE]
-                          SHUFFLE [RS_37]
-                            PartitionCols:_col2
-                            Merge Join Operator [MERGEJOIN_83] (rows=210834322 width=135)
-                              Conds:RS_34._col1=RS_35._col0(Inner),Output:["_col2","_col3","_col4","_col5"]
-                            <-Map 12 [SIMPLE_EDGE]
-                              SHUFFLE [RS_35]
-                                PartitionCols:_col0
-                                Select Operator [SEL_19] (rows=20000000 width=1014)
-                                  Output:["_col0"]
-                                  Filter Operator [FIL_77] (rows=20000000 width=1014)
-                                    predicate:((ca_state = 'TX') and ca_address_sk is not null)
-                                    TableScan [TS_17] (rows=40000000 width=1014)
-                                      default@customer_address,ca,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_state"]
-                            <-Reducer 3 [SIMPLE_EDGE]
-                              SHUFFLE [RS_34]
-                                PartitionCols:_col1
-                                Merge Join Operator [MERGEJOIN_82] (rows=191667562 width=135)
-                                  Conds:RS_31._col0=RS_32._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
-                                <-Map 11 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_32]
+          Select Operator [SEL_71] (rows=1 width=344)
+            Output:["_col0","_col1","_col2"]
+          <-Reducer 7 [SIMPLE_EDGE]
+            SHUFFLE [RS_70]
+              Select Operator [SEL_69] (rows=1 width=344)
+                Output:["_col1","_col2","_col3"]
+                Group By Operator [GBY_68] (rows=1 width=344)
+                  Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT KEY._col0:0._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
+                <-Reducer 6 [SIMPLE_EDGE]
+                  SHUFFLE [RS_67]
+                    Group By Operator [GBY_66] (rows=1395035081047425024 width=1)
+                      Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT _col4)","sum(_col5)","sum(_col6)"],keys:_col4
+                      Select Operator [SEL_65] (rows=1395035081047425024 width=1)
+                        Output:["_col4","_col5","_col6"]
+                        Filter Operator [FIL_64] (rows=1395035081047425024 width=1)
+                          predicate:_col16 is null
+                          Select Operator [SEL_63] (rows=2790070162094850048 width=1)
+                            Output:["_col4","_col5","_col6","_col16"]
+                            Merge Join Operator [MERGEJOIN_113] (rows=2790070162094850048 width=1)
+                              Conds:RS_60._col3, _col4=RS_61._col0, _col1(Inner),Output:["_col4","_col5","_col6","_col14"]
+                            <-Reducer 16 [SIMPLE_EDGE]
+                              SHUFFLE [RS_61]
+                                PartitionCols:_col0, _col1
+                                Group By Operator [GBY_46] (rows=2536427365110644736 width=1)
+                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                <-Reducer 15 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_45]
+                                    PartitionCols:_col0, _col1
+                                    Group By Operator [GBY_44] (rows=5072854730221289472 width=1)
+                                      Output:["_col0","_col1"],keys:_col2, _col3
+                                      Select Operator [SEL_43] (rows=5072854730221289472 width=1)
+                                        Output:["_col2","_col3"]
+                                        Filter Operator [FIL_42] (rows=5072854730221289472 width=1)
+                                          predicate:(_col2 <> _col0)
+                                          Merge Join Operator [MERGEJOIN_111] (rows=5072854730221289472 width=1)
+                                            Conds:RS_39._col1=RS_40._col1(Inner),Output:["_col0","_col2","_col3"]
+                                          <-Map 14 [SIMPLE_EDGE]
+                                            PARTITION_ONLY_SHUFFLE [RS_39]
+                                              PartitionCols:_col1
+                                              Select Operator [SEL_20] (rows=144002668 width=135)
+                                                Output:["_col0","_col1"]
+                                                TableScan [TS_19] (rows=144002668 width=135)
+                                                  default@web_sales,ws2,Tbl:COMPLETE,Col:NONE,Output:["ws_warehouse_sk","ws_order_number"]
+                                          <-Reducer 18 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_40]
+                                              PartitionCols:_col1
+                                              Select Operator [SEL_38] (rows=4611686018427387903 width=1)
+                                                Output:["_col0","_col1"]
+                                                Group By Operator [GBY_37] (rows=4611686018427387903 width=1)
+                                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                                <-Reducer 17 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_36]
+                                                    PartitionCols:_col0, _col1
+                                                    Group By Operator [GBY_35] (rows=9223372036854775807 width=1)
+                                                      Output:["_col0","_col1"],keys:_col4, _col3
+                                                      Merge Join Operator [MERGEJOIN_107] (rows=9223372036854775807 width=1)
+                                                        Conds:(Inner),(Inner),(Inner),Output:["_col3","_col4"]
+                                                      <-Map 14 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_32]
+                                                          Select Operator [SEL_28] (rows=144002668 width=135)
+                                                            Output:["_col0","_col1"]
+                                                             Please refer to the previous TableScan [TS_19]
+                                                      <-Map 19 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_29]
+                                                          Select Operator [SEL_22] (rows=73049 width=4)
+                                                            TableScan [TS_21] (rows=73049 width=1119)
+                                                              default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE
+                                                      <-Map 20 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_30]
+                                                          Select Operator [SEL_24] (rows=84 width=4)
+                                                            TableScan [TS_23] (rows=84 width=1850)
+                                                              default@web_site,web_site,Tbl:COMPLETE,Col:COMPLETE
+                                                      <-Map 21 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_31]
+                                                          Select Operator [SEL_26] (rows=40000000 width=4)
+                                                            TableScan [TS_25] (rows=40000000 width=1014)
+                                                              default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE
+                            <-Reducer 5 [SIMPLE_EDGE]
+                              SHUFFLE [RS_60]
+                                PartitionCols:_col3, _col4
+                                Merge Join Operator [MERGEJOIN_112] (rows=210834322 width=135)
+                                  Conds:RS_57._col4=RS_58._col0(Left Outer),Output:["_col3","_col4","_col5","_col6","_col14"]
+                                <-Reducer 13 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_58]
                                     PartitionCols:_col0
-                                    Select Operator [SEL_16] (rows=8116 width=1119)
-                                      Output:["_col0"]
-                                      Filter Operator [FIL_76] (rows=8116 width=1119)
-                                        predicate:(d_date BETWEEN '1999-05-01' AND '1999-07-01' and d_date_sk is not null)
-                                        TableScan [TS_14] (rows=73049 width=1119)
-                                          default@date_dim,d,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
-                                <-Reducer 2 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_31]
-                                    PartitionCols:_col0
-                                    Merge Join Operator [MERGEJOIN_81] (rows=174243235 width=135)
-                                      Conds:RS_28._col3=RS_29._col0(Left Semi),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                    <-Map 1 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_28]
-                                        PartitionCols:_col3
-                                        Select Operator [SEL_2] (rows=144002668 width=135)
-                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                          Filter Operator [FIL_73] (rows=144002668 width=135)
-                                            predicate:(ws_ship_addr_sk is not null and ws_web_site_sk is not null and ws_ship_date_sk is not null and ws_order_number is not null)
-                                            TableScan [TS_0] (rows=144002668 width=135)
-                                              default@web_sales,ws1,Tbl:COMPLETE,Col:NONE,Output:["ws_ship_date_sk","ws_ship_addr_sk","ws_web_site_sk","ws_order_number","ws_ext_ship_cost","ws_net_profit"]
-                                    <-Reducer 9 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_29]
+                                    Select Operator [SEL_18] (rows=7199233 width=92)
+                                      Output:["_col0","_col1"]
+                                      Group By Operator [GBY_17] (rows=7199233 width=92)
+                                        Output:["_col0"],keys:KEY._col0
+                                      <-Map 12 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_16]
+                                          PartitionCols:_col0
+                                          Group By Operator [GBY_15] (rows=14398467 width=92)
+                                            Output:["_col0"],keys:wr_order_number
+                                            Filter Operator [FIL_104] (rows=14398467 width=92)
+                                              predicate:wr_order_number is not null
+                                              TableScan [TS_12] (rows=14398467 width=92)
+                                                default@web_returns,wr1,Tbl:COMPLETE,Col:NONE,Output:["wr_order_number"]
+                                <-Reducer 4 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_57]
+                                    PartitionCols:_col4
+                                    Merge Join Operator [MERGEJOIN_110] (rows=191667562 width=135)
+                                      Conds:RS_54._col2=RS_55._col0(Inner),Output:["_col3","_col4","_col5","_col6"]
+                                    <-Map 11 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_55]
                                         PartitionCols:_col0
-                                        Group By Operator [GBY_27] (rows=158402938 width=135)
-                                          Output:["_col0"],keys:_col0
-                                          Select Operator [SEL_13] (rows=158402938 width=135)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_12] (rows=158402938 width=135)
-                                              predicate:(_col0 <> _col2)
-                                              Merge Join Operator [MERGEJOIN_80] (rows=158402938 width=135)
-                                                Conds:RS_9._col1=RS_10._col1(Inner),Output:["_col0","_col1","_col2"]
-                                              <-Map 10 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_10]
-                                                  PartitionCols:_col1
-                                                  Select Operator [SEL_8] (rows=144002668 width=135)
-                                                    Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_75] (rows=144002668 width=135)
-                                                      predicate:ws_order_number is not null
-                                                      TableScan [TS_6] (rows=144002668 width=135)
-                                                        default@web_sales,ws3,Tbl:COMPLETE,Col:NONE,Output:["ws_warehouse_sk","ws_order_number"]
-                                              <-Map 8 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_9]
-                                                  PartitionCols:_col1
-                                                  Select Operator [SEL_5] (rows=144002668 width=135)
-                                                    Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_74] (rows=144002668 width=135)
-                                                      predicate:ws_order_number is not null
-                                                      TableScan [TS_3] (rows=144002668 width=135)
-                                                        default@web_sales,ws2,Tbl:COMPLETE,Col:NONE,Output:["ws_warehouse_sk","ws_order_number"]
+                                        Select Operator [SEL_11] (rows=42 width=1850)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_103] (rows=42 width=1850)
+                                            predicate:((web_company_name = 'pri') and web_site_sk is not null)
+                                            TableScan [TS_9] (rows=84 width=1850)
+                                              default@web_site,web_site,Tbl:COMPLETE,Col:NONE,Output:["web_site_sk","web_company_name"]
+                                    <-Reducer 3 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_54]
+                                        PartitionCols:_col2
+                                        Merge Join Operator [MERGEJOIN_109] (rows=174243235 width=135)
+                                          Conds:RS_51._col1=RS_52._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col6"]
+                                        <-Map 10 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_52]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_8] (rows=20000000 width=1014)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_102] (rows=20000000 width=1014)
+                                                predicate:((ca_state = 'TX') and ca_address_sk is not null)
+                                                TableScan [TS_6] (rows=40000000 width=1014)
+                                                  default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_state"]
+                                        <-Reducer 2 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_51]
+                                            PartitionCols:_col1
+                                            Merge Join Operator [MERGEJOIN_108] (rows=158402938 width=135)
+                                              Conds:RS_48._col0=RS_49._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
+                                            <-Map 1 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_48]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_2] (rows=144002668 width=135)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                  Filter Operator [FIL_100] (rows=144002668 width=135)
+                                                    predicate:(ws_ship_date_sk is not null and ws_ship_addr_sk is not null and ws_web_site_sk is not null)
+                                                    TableScan [TS_0] (rows=144002668 width=135)
+                                                      default@web_sales,ws1,Tbl:COMPLETE,Col:NONE,Output:["ws_ship_date_sk","ws_ship_addr_sk","ws_web_site_sk","ws_warehouse_sk","ws_order_number","ws_ext_ship_cost","ws_net_profit"]
+                                            <-Map 9 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_49]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_5] (rows=8116 width=1119)
+                                                  Output:["_col0"]
+                                                  Filter Operator [FIL_101] (rows=8116 width=1119)
+                                                    predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1999-05-01 00:00:00.0 AND 1999-06-30 00:00:00.0 and d_date_sk is not null)
+                                                    TableScan [TS_3] (rows=73049 width=1119)
+                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
 


[08/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query58.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query58.q.out b/ql/src/test/results/clientpositive/perf/query58.q.out
index e73605a..acdfc07 100644
--- a/ql/src/test/results/clientpositive/perf/query58.q.out
+++ b/ql/src/test/results/clientpositive/perf/query58.q.out
@@ -1,4 +1,50 @@
-PREHOOK: query: explain select  ss_items.item_id
+Warning: Shuffle Join MERGEJOIN[265][tables = [$hdt$_4, $hdt$_5]] in Stage 'Reducer 20' is a cross product
+Warning: Shuffle Join MERGEJOIN[269][tables = [$hdt$_5, $hdt$_6]] in Stage 'Reducer 24' is a cross product
+Warning: Shuffle Join MERGEJOIN[273][tables = [$hdt$_6, $hdt$_7]] in Stage 'Reducer 28' is a cross product
+PREHOOK: query: explain
+with ss_items as
+ (select i_item_id item_id
+        ,sum(ss_ext_sales_price) ss_item_rev 
+ from store_sales
+     ,item
+     ,date_dim
+ where ss_item_sk = i_item_sk
+   and d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq 
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+   and ss_sold_date_sk   = d_date_sk
+ group by i_item_id),
+ cs_items as
+ (select i_item_id item_id
+        ,sum(cs_ext_sales_price) cs_item_rev
+  from catalog_sales
+      ,item
+      ,date_dim
+ where cs_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq 
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+  and  cs_sold_date_sk = d_date_sk
+ group by i_item_id),
+ ws_items as
+ (select i_item_id item_id
+        ,sum(ws_ext_sales_price) ws_item_rev
+  from web_sales
+      ,item
+      ,date_dim
+ where ws_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq =(select d_week_seq 
+                                     from date_dim
+                                     where d_date = '1998-02-19'))
+  and ws_sold_date_sk   = d_date_sk
+ group by i_item_id)
+  select  ss_items.item_id
        ,ss_item_rev
        ,ss_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 ss_dev
        ,cs_item_rev
@@ -6,46 +52,63 @@ PREHOOK: query: explain select  ss_items.item_id
        ,ws_item_rev
        ,ws_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 ws_dev
        ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average
-FROM
-( select i_item_id item_id ,sum(ss_ext_sales_price) as ss_item_rev 
- from store_sales
-     JOIN item ON store_sales.ss_item_sk = item.i_item_sk
-     JOIN date_dim ON store_sales.ss_sold_date_sk = date_dim.d_date_sk
-     JOIN (select d1.d_date
-                 from date_dim d1 JOIN date_dim d2 ON d1.d_week_seq = d2.d_week_seq
-                 where d2.d_date = '1998-08-04') sub ON date_dim.d_date = sub.d_date
- group by i_item_id ) ss_items
-JOIN
-( select i_item_id item_id ,sum(cs_ext_sales_price) as cs_item_rev 
- from catalog_sales
-     JOIN item ON catalog_sales.cs_item_sk = item.i_item_sk
-     JOIN date_dim ON catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
-     JOIN (select d1.d_date
-                 from date_dim d1 JOIN date_dim d2 ON d1.d_week_seq = d2.d_week_seq
-                 where d2.d_date = '1998-08-04') sub ON date_dim.d_date = sub.d_date
- group by i_item_id ) cs_items
-ON ss_items.item_id=cs_items.item_id
-JOIN
-( select i_item_id item_id ,sum(ws_ext_sales_price) as ws_item_rev 
- from web_sales
-     JOIN item ON web_sales.ws_item_sk = item.i_item_sk
-     JOIN date_dim ON web_sales.ws_sold_date_sk = date_dim.d_date_sk
-     JOIN (select d1.d_date
-                 from date_dim d1 JOIN date_dim d2 ON d1.d_week_seq = d2.d_week_seq
-                 where d2.d_date = '1998-08-04') sub ON date_dim.d_date = sub.d_date
- group by i_item_id ) ws_items
-ON ss_items.item_id=ws_items.item_id 
- where
-       ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+ from ss_items,cs_items,ws_items
+ where ss_items.item_id=cs_items.item_id
+   and ss_items.item_id=ws_items.item_id 
+   and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
    and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
    and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
    and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
    and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
    and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
- order by item_id ,ss_item_rev
+ order by item_id
+         ,ss_item_rev
  limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select  ss_items.item_id
+POSTHOOK: query: explain
+with ss_items as
+ (select i_item_id item_id
+        ,sum(ss_ext_sales_price) ss_item_rev 
+ from store_sales
+     ,item
+     ,date_dim
+ where ss_item_sk = i_item_sk
+   and d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq 
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+   and ss_sold_date_sk   = d_date_sk
+ group by i_item_id),
+ cs_items as
+ (select i_item_id item_id
+        ,sum(cs_ext_sales_price) cs_item_rev
+  from catalog_sales
+      ,item
+      ,date_dim
+ where cs_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq 
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+  and  cs_sold_date_sk = d_date_sk
+ group by i_item_id),
+ ws_items as
+ (select i_item_id item_id
+        ,sum(ws_ext_sales_price) ws_item_rev
+  from web_sales
+      ,item
+      ,date_dim
+ where ws_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq =(select d_week_seq 
+                                     from date_dim
+                                     where d_date = '1998-02-19'))
+  and ws_sold_date_sk   = d_date_sk
+ group by i_item_id)
+  select  ss_items.item_id
        ,ss_item_rev
        ,ss_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 ss_dev
        ,cs_item_rev
@@ -53,290 +116,351 @@ POSTHOOK: query: explain select  ss_items.item_id
        ,ws_item_rev
        ,ws_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 ws_dev
        ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average
-FROM
-( select i_item_id item_id ,sum(ss_ext_sales_price) as ss_item_rev 
- from store_sales
-     JOIN item ON store_sales.ss_item_sk = item.i_item_sk
-     JOIN date_dim ON store_sales.ss_sold_date_sk = date_dim.d_date_sk
-     JOIN (select d1.d_date
-                 from date_dim d1 JOIN date_dim d2 ON d1.d_week_seq = d2.d_week_seq
-                 where d2.d_date = '1998-08-04') sub ON date_dim.d_date = sub.d_date
- group by i_item_id ) ss_items
-JOIN
-( select i_item_id item_id ,sum(cs_ext_sales_price) as cs_item_rev 
- from catalog_sales
-     JOIN item ON catalog_sales.cs_item_sk = item.i_item_sk
-     JOIN date_dim ON catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
-     JOIN (select d1.d_date
-                 from date_dim d1 JOIN date_dim d2 ON d1.d_week_seq = d2.d_week_seq
-                 where d2.d_date = '1998-08-04') sub ON date_dim.d_date = sub.d_date
- group by i_item_id ) cs_items
-ON ss_items.item_id=cs_items.item_id
-JOIN
-( select i_item_id item_id ,sum(ws_ext_sales_price) as ws_item_rev 
- from web_sales
-     JOIN item ON web_sales.ws_item_sk = item.i_item_sk
-     JOIN date_dim ON web_sales.ws_sold_date_sk = date_dim.d_date_sk
-     JOIN (select d1.d_date
-                 from date_dim d1 JOIN date_dim d2 ON d1.d_week_seq = d2.d_week_seq
-                 where d2.d_date = '1998-08-04') sub ON date_dim.d_date = sub.d_date
- group by i_item_id ) ws_items
-ON ss_items.item_id=ws_items.item_id 
- where
-       ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+ from ss_items,cs_items,ws_items
+ where ss_items.item_id=cs_items.item_id
+   and ss_items.item_id=ws_items.item_id 
+   and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
    and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
    and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
    and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
    and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
    and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
- order by item_id ,ss_item_rev
+ order by item_id
+         ,ss_item_rev
  limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 10 <- Map 22 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 11 <- Reducer 10 (SIMPLE_EDGE)
-Reducer 12 <- Map 1 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE)
-Reducer 13 <- Map 24 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE)
-Reducer 14 <- Map 22 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE)
-Reducer 15 <- Reducer 14 (SIMPLE_EDGE)
-Reducer 17 <- Map 16 (SIMPLE_EDGE), Map 20 (SIMPLE_EDGE)
-Reducer 18 <- Map 16 (SIMPLE_EDGE), Map 20 (SIMPLE_EDGE)
-Reducer 19 <- Map 16 (SIMPLE_EDGE), Map 20 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 17 (SIMPLE_EDGE)
-Reducer 3 <- Map 21 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 22 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-Reducer 6 <- Reducer 11 (SIMPLE_EDGE), Reducer 15 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
-Reducer 8 <- Map 1 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE)
-Reducer 9 <- Map 23 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+Reducer 10 <- Reducer 9 (SIMPLE_EDGE)
+Reducer 11 <- Map 33 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 17 (SIMPLE_EDGE)
+Reducer 13 <- Reducer 12 (SIMPLE_EDGE)
+Reducer 15 <- Map 14 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE)
+Reducer 16 <- Map 14 (SIMPLE_EDGE), Reducer 26 (SIMPLE_EDGE)
+Reducer 17 <- Map 14 (SIMPLE_EDGE), Reducer 30 (SIMPLE_EDGE)
+Reducer 19 <- Map 18 (CUSTOM_SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 20 <- Map 31 (CUSTOM_SIMPLE_EDGE), Reducer 19 (CUSTOM_SIMPLE_EDGE)
+Reducer 21 <- Map 31 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE)
+Reducer 22 <- Reducer 21 (SIMPLE_EDGE)
+Reducer 23 <- Map 18 (CUSTOM_SIMPLE_EDGE)
+Reducer 24 <- Map 31 (CUSTOM_SIMPLE_EDGE), Reducer 23 (CUSTOM_SIMPLE_EDGE)
+Reducer 25 <- Map 31 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
+Reducer 26 <- Reducer 25 (SIMPLE_EDGE)
+Reducer 27 <- Map 18 (CUSTOM_SIMPLE_EDGE)
+Reducer 28 <- Map 31 (CUSTOM_SIMPLE_EDGE), Reducer 27 (CUSTOM_SIMPLE_EDGE)
+Reducer 29 <- Map 31 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 15 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 30 <- Reducer 29 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 10 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 8 <- Map 32 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 16 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:100
     Stage-1
-      Reducer 7
-      File Output Operator [FS_113]
-        Limit [LIM_112] (rows=100 width=88)
+      Reducer 6
+      File Output Operator [FS_164]
+        Limit [LIM_163] (rows=100 width=88)
           Number of rows:100
-          Select Operator [SEL_111] (rows=1442 width=88)
+          Select Operator [SEL_162] (rows=1442 width=88)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
-          <-Reducer 6 [SIMPLE_EDGE]
-            SHUFFLE [RS_110]
-              Select Operator [SEL_109] (rows=1442 width=88)
+          <-Reducer 5 [SIMPLE_EDGE]
+            SHUFFLE [RS_161]
+              Select Operator [SEL_160] (rows=1442 width=88)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
-                Filter Operator [FIL_103] (rows=1442 width=88)
+                Filter Operator [FIL_154] (rows=1442 width=88)
                   predicate:(_col5 BETWEEN (0.9 * _col1) AND (1.1 * _col1) and _col5 BETWEEN (0.9 * _col3) AND (1.1 * _col3) and _col1 BETWEEN (0.9 * _col5) AND (1.1 * _col5) and _col3 BETWEEN (0.9 * _col5) AND (1.1 * _col5) and _col1 BETWEEN (0.9 * _col3) AND (1.1 * _col3) and _col3 BETWEEN (0.9 * _col1) AND (1.1 * _col1))
-                  Merge Join Operator [MERGEJOIN_213] (rows=766650239 width=88)
-                    Conds:RS_99._col0=RS_100._col0(Inner),RS_99._col0=RS_101._col0(Inner),Output:["_col0","_col1","_col3","_col5"]
-                  <-Reducer 11 [SIMPLE_EDGE]
-                    SHUFFLE [RS_100]
+                  Merge Join Operator [MERGEJOIN_279] (rows=766650239 width=88)
+                    Conds:RS_150._col0=RS_151._col0(Inner),RS_150._col0=RS_152._col0(Inner),Output:["_col0","_col1","_col3","_col5"]
+                  <-Reducer 10 [SIMPLE_EDGE]
+                    SHUFFLE [RS_151]
                       PartitionCols:_col0
-                      Group By Operator [GBY_64] (rows=348477374 width=88)
+                      Group By Operator [GBY_98] (rows=348477374 width=88)
                         Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                      <-Reducer 10 [SIMPLE_EDGE]
-                        SHUFFLE [RS_63]
+                      <-Reducer 9 [SIMPLE_EDGE]
+                        SHUFFLE [RS_97]
                           PartitionCols:_col0
-                          Group By Operator [GBY_62] (rows=696954748 width=88)
-                            Output:["_col0","_col1"],aggregations:["sum(_col5)"],keys:_col7
-                            Merge Join Operator [MERGEJOIN_208] (rows=696954748 width=88)
-                              Conds:RS_58._col4=RS_59._col0(Inner),Output:["_col5","_col7"]
-                            <-Map 22 [SIMPLE_EDGE]
-                              SHUFFLE [RS_59]
+                          Group By Operator [GBY_96] (rows=696954748 width=88)
+                            Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col4
+                            Merge Join Operator [MERGEJOIN_277] (rows=696954748 width=88)
+                              Conds:RS_92._col0=RS_93._col0(Inner),Output:["_col2","_col4"]
+                            <-Reducer 16 [SIMPLE_EDGE]
+                              SHUFFLE [RS_93]
+                                PartitionCols:_col0
+                                Merge Join Operator [MERGEJOIN_271] (rows=80353 width=1119)
+                                  Conds:RS_85._col1=RS_86._col0(Inner),Output:["_col0"]
+                                <-Map 14 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_85]
+                                    PartitionCols:_col1
+                                    Select Operator [SEL_58] (rows=73049 width=1119)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_254] (rows=73049 width=1119)
+                                        predicate:(d_date is not null 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_date"]
+                                <-Reducer 26 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_86]
+                                    PartitionCols:_col0
+                                    Group By Operator [GBY_83] (rows=40176 width=1119)
+                                      Output:["_col0"],keys:KEY._col0
+                                    <-Reducer 25 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_82]
+                                        PartitionCols:_col0
+                                        Group By Operator [GBY_81] (rows=80353 width=1119)
+                                          Output:["_col0"],keys:_col2
+                                          Merge Join Operator [MERGEJOIN_270] (rows=80353 width=1119)
+                                            Conds:RS_77._col1=RS_78._col1(Inner),Output:["_col2"]
+                                          <-Map 31 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_78]
+                                              PartitionCols:_col1
+                                              Select Operator [SEL_73] (rows=73049 width=1119)
+                                                Output:["_col0","_col1"]
+                                                Filter Operator [FIL_257] (rows=73049 width=1119)
+                                                  predicate:(d_week_seq is not null and d_date is not null)
+                                                  TableScan [TS_18] (rows=73049 width=1119)
+                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date","d_week_seq"]
+                                          <-Reducer 24 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_77]
+                                              PartitionCols:_col1
+                                              Merge Join Operator [MERGEJOIN_269] (rows=36524 width=1128)
+                                                Conds:(Inner),Output:["_col1"]
+                                              <-Map 31 [CUSTOM_SIMPLE_EDGE]
+                                                SHUFFLE [RS_75]
+                                                  Select Operator [SEL_70] (rows=36524 width=1119)
+                                                    Output:["_col0"]
+                                                    Filter Operator [FIL_256] (rows=36524 width=1119)
+                                                      predicate:((d_date = '1998-02-19') and d_week_seq is not null)
+                                                       Please refer to the previous TableScan [TS_18]
+                                              <-Reducer 23 [CUSTOM_SIMPLE_EDGE]
+                                                PARTITION_ONLY_SHUFFLE [RS_74]
+                                                  Select Operator [SEL_67] (rows=1 width=8)
+                                                    Filter Operator [FIL_66] (rows=1 width=8)
+                                                      predicate:(sq_count_check(_col0) <= 1)
+                                                      Group By Operator [GBY_64] (rows=1 width=8)
+                                                        Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                                                      <-Map 18 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_63]
+                                                          Group By Operator [GBY_62] (rows=1 width=8)
+                                                            Output:["_col0"],aggregations:["count()"]
+                                                            Select Operator [SEL_61] (rows=36524 width=1119)
+                                                              Filter Operator [FIL_255] (rows=36524 width=1119)
+                                                                predicate:(d_date = '1998-02-19')
+                                                                TableScan [TS_9] (rows=73049 width=1119)
+                                                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date"]
+                            <-Reducer 8 [SIMPLE_EDGE]
+                              SHUFFLE [RS_92]
                                 PartitionCols:_col0
-                                Select Operator [SEL_51] (rows=462000 width=1436)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_195] (rows=462000 width=1436)
-                                    predicate:(i_item_sk is not null and i_item_id is not null)
-                                    TableScan [TS_16] (rows=462000 width=1436)
-                                      default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_item_id"]
-                            <-Reducer 9 [SIMPLE_EDGE]
-                              SHUFFLE [RS_58]
-                                PartitionCols:_col4
-                                Merge Join Operator [MERGEJOIN_207] (rows=633595212 width=88)
-                                  Conds:RS_55._col0=RS_56._col0(Inner),Output:["_col4","_col5"]
-                                <-Map 23 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_56]
+                                Merge Join Operator [MERGEJOIN_268] (rows=633595212 width=88)
+                                  Conds:RS_89._col1=RS_90._col0(Inner),Output:["_col0","_col2","_col4"]
+                                <-Map 7 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_90]
                                     PartitionCols:_col0
-                                    Select Operator [SEL_48] (rows=575995635 width=88)
+                                    Select Operator [SEL_55] (rows=462000 width=1436)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_253] (rows=462000 width=1436)
+                                        predicate:(i_item_sk is not null and i_item_id is not null)
+                                        TableScan [TS_3] (rows=462000 width=1436)
+                                          default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_item_id"]
+                                <-Map 32 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_89]
+                                    PartitionCols:_col1
+                                    Select Operator [SEL_52] (rows=575995635 width=88)
                                       Output:["_col0","_col1","_col2"]
-                                      Filter Operator [FIL_194] (rows=575995635 width=88)
+                                      Filter Operator [FIL_252] (rows=575995635 width=88)
                                         predicate:(ss_item_sk is not null and ss_sold_date_sk is not null)
-                                        TableScan [TS_46] (rows=575995635 width=88)
+                                        TableScan [TS_50] (rows=575995635 width=88)
                                           default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ext_sales_price"]
-                                <-Reducer 8 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_55]
-                                    PartitionCols:_col0
-                                    Merge Join Operator [MERGEJOIN_206] (rows=88388 width=1119)
-                                      Conds:RS_52._col1=RS_53._col0(Inner),Output:["_col0"]
-                                    <-Map 1 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_52]
-                                        PartitionCols:_col1
-                                        Select Operator [SEL_35] (rows=73049 width=1119)
-                                          Output:["_col0","_col1"]
-                                          Filter Operator [FIL_191] (rows=73049 width=1119)
-                                            predicate:(d_date_sk is not null and d_date is not null)
-                                            TableScan [TS_0] (rows=73049 width=1119)
-                                              default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
-                                    <-Reducer 18 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_53]
-                                        PartitionCols:_col0
-                                        Merge Join Operator [MERGEJOIN_205] (rows=80353 width=1119)
-                                          Conds:RS_42._col1=RS_43._col1(Inner),Output:["_col0"]
-                                        <-Map 16 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_42]
-                                            PartitionCols:_col1
-                                            Select Operator [SEL_38] (rows=73049 width=1119)
-                                              Output:["_col0","_col1"]
-                                              Filter Operator [FIL_192] (rows=73049 width=1119)
-                                                predicate:(d_week_seq is not null and d_date is not null)
-                                                TableScan [TS_3] (rows=73049 width=1119)
-                                                  default@date_dim,d1,Tbl:COMPLETE,Col:NONE,Output:["d_date","d_week_seq"]
-                                        <-Map 20 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_43]
-                                            PartitionCols:_col1
-                                            Select Operator [SEL_41] (rows=36524 width=1119)
-                                              Output:["_col1"]
-                                              Filter Operator [FIL_193] (rows=36524 width=1119)
-                                                predicate:((d_date = '1998-08-04') and d_week_seq is not null)
-                                                TableScan [TS_6] (rows=73049 width=1119)
-                                                  default@date_dim,d2,Tbl:COMPLETE,Col:NONE,Output:["d_date","d_week_seq"]
-                  <-Reducer 15 [SIMPLE_EDGE]
-                    SHUFFLE [RS_101]
+                  <-Reducer 13 [SIMPLE_EDGE]
+                    SHUFFLE [RS_152]
                       PartitionCols:_col0
-                      Group By Operator [GBY_97] (rows=87121617 width=135)
+                      Group By Operator [GBY_148] (rows=87121617 width=135)
                         Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                      <-Reducer 14 [SIMPLE_EDGE]
-                        SHUFFLE [RS_96]
+                      <-Reducer 12 [SIMPLE_EDGE]
+                        SHUFFLE [RS_147]
                           PartitionCols:_col0
-                          Group By Operator [GBY_95] (rows=174243235 width=135)
-                            Output:["_col0","_col1"],aggregations:["sum(_col5)"],keys:_col7
-                            Merge Join Operator [MERGEJOIN_212] (rows=174243235 width=135)
-                              Conds:RS_91._col4=RS_92._col0(Inner),Output:["_col5","_col7"]
-                            <-Map 22 [SIMPLE_EDGE]
-                              SHUFFLE [RS_92]
+                          Group By Operator [GBY_146] (rows=174243235 width=135)
+                            Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col4
+                            Merge Join Operator [MERGEJOIN_278] (rows=174243235 width=135)
+                              Conds:RS_142._col0=RS_143._col0(Inner),Output:["_col2","_col4"]
+                            <-Reducer 11 [SIMPLE_EDGE]
+                              SHUFFLE [RS_142]
                                 PartitionCols:_col0
-                                Select Operator [SEL_84] (rows=462000 width=1436)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_200] (rows=462000 width=1436)
-                                    predicate:(i_item_sk is not null and i_item_id is not null)
-                                     Please refer to the previous TableScan [TS_16]
-                            <-Reducer 13 [SIMPLE_EDGE]
-                              SHUFFLE [RS_91]
-                                PartitionCols:_col4
-                                Merge Join Operator [MERGEJOIN_211] (rows=158402938 width=135)
-                                  Conds:RS_88._col0=RS_89._col0(Inner),Output:["_col4","_col5"]
-                                <-Map 24 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_89]
+                                Merge Join Operator [MERGEJOIN_272] (rows=158402938 width=135)
+                                  Conds:RS_139._col1=RS_140._col0(Inner),Output:["_col0","_col2","_col4"]
+                                <-Map 7 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_140]
                                     PartitionCols:_col0
-                                    Select Operator [SEL_81] (rows=144002668 width=135)
+                                    Select Operator [SEL_105] (rows=462000 width=1436)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_259] (rows=462000 width=1436)
+                                        predicate:(i_item_sk is not null and i_item_id is not null)
+                                         Please refer to the previous TableScan [TS_3]
+                                <-Map 33 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_139]
+                                    PartitionCols:_col1
+                                    Select Operator [SEL_102] (rows=144002668 width=135)
                                       Output:["_col0","_col1","_col2"]
-                                      Filter Operator [FIL_199] (rows=144002668 width=135)
+                                      Filter Operator [FIL_258] (rows=144002668 width=135)
                                         predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
-                                        TableScan [TS_79] (rows=144002668 width=135)
+                                        TableScan [TS_100] (rows=144002668 width=135)
                                           default@web_sales,web_sales,Tbl:COMPLETE,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_ext_sales_price"]
-                                <-Reducer 12 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_88]
+                            <-Reducer 17 [SIMPLE_EDGE]
+                              SHUFFLE [RS_143]
+                                PartitionCols:_col0
+                                Merge Join Operator [MERGEJOIN_275] (rows=80353 width=1119)
+                                  Conds:RS_135._col1=RS_136._col0(Inner),Output:["_col0"]
+                                <-Map 14 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_135]
+                                    PartitionCols:_col1
+                                    Select Operator [SEL_108] (rows=73049 width=1119)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_260] (rows=73049 width=1119)
+                                        predicate:(d_date is not null and d_date_sk is not null)
+                                         Please refer to the previous TableScan [TS_6]
+                                <-Reducer 30 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_136]
                                     PartitionCols:_col0
-                                    Merge Join Operator [MERGEJOIN_210] (rows=88388 width=1119)
-                                      Conds:RS_85._col1=RS_86._col0(Inner),Output:["_col0"]
-                                    <-Map 1 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_85]
-                                        PartitionCols:_col1
-                                        Select Operator [SEL_68] (rows=73049 width=1119)
-                                          Output:["_col0","_col1"]
-                                          Filter Operator [FIL_196] (rows=73049 width=1119)
-                                            predicate:(d_date_sk is not null and d_date is not null)
-                                             Please refer to the previous TableScan [TS_0]
-                                    <-Reducer 19 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_86]
+                                    Group By Operator [GBY_133] (rows=40176 width=1119)
+                                      Output:["_col0"],keys:KEY._col0
+                                    <-Reducer 29 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_132]
                                         PartitionCols:_col0
-                                        Merge Join Operator [MERGEJOIN_209] (rows=80353 width=1119)
-                                          Conds:RS_75._col1=RS_76._col1(Inner),Output:["_col0"]
-                                        <-Map 16 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_75]
-                                            PartitionCols:_col1
-                                            Select Operator [SEL_71] (rows=73049 width=1119)
-                                              Output:["_col0","_col1"]
-                                              Filter Operator [FIL_197] (rows=73049 width=1119)
-                                                predicate:(d_week_seq is not null and d_date is not null)
-                                                 Please refer to the previous TableScan [TS_3]
-                                        <-Map 20 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_76]
-                                            PartitionCols:_col1
-                                            Select Operator [SEL_74] (rows=36524 width=1119)
-                                              Output:["_col1"]
-                                              Filter Operator [FIL_198] (rows=36524 width=1119)
-                                                predicate:((d_date = '1998-08-04') and d_week_seq is not null)
-                                                 Please refer to the previous TableScan [TS_6]
-                  <-Reducer 5 [SIMPLE_EDGE]
-                    SHUFFLE [RS_99]
+                                        Group By Operator [GBY_131] (rows=80353 width=1119)
+                                          Output:["_col0"],keys:_col2
+                                          Merge Join Operator [MERGEJOIN_274] (rows=80353 width=1119)
+                                            Conds:RS_127._col1=RS_128._col1(Inner),Output:["_col2"]
+                                          <-Map 31 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_128]
+                                              PartitionCols:_col1
+                                              Select Operator [SEL_123] (rows=73049 width=1119)
+                                                Output:["_col0","_col1"]
+                                                Filter Operator [FIL_263] (rows=73049 width=1119)
+                                                  predicate:(d_week_seq is not null and d_date is not null)
+                                                   Please refer to the previous TableScan [TS_18]
+                                          <-Reducer 28 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_127]
+                                              PartitionCols:_col1
+                                              Merge Join Operator [MERGEJOIN_273] (rows=36524 width=1128)
+                                                Conds:(Inner),Output:["_col1"]
+                                              <-Map 31 [CUSTOM_SIMPLE_EDGE]
+                                                SHUFFLE [RS_125]
+                                                  Select Operator [SEL_120] (rows=36524 width=1119)
+                                                    Output:["_col0"]
+                                                    Filter Operator [FIL_262] (rows=36524 width=1119)
+                                                      predicate:((d_date = '1998-02-19') and d_week_seq is not null)
+                                                       Please refer to the previous TableScan [TS_18]
+                                              <-Reducer 27 [CUSTOM_SIMPLE_EDGE]
+                                                PARTITION_ONLY_SHUFFLE [RS_124]
+                                                  Select Operator [SEL_117] (rows=1 width=8)
+                                                    Filter Operator [FIL_116] (rows=1 width=8)
+                                                      predicate:(sq_count_check(_col0) <= 1)
+                                                      Group By Operator [GBY_114] (rows=1 width=8)
+                                                        Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                                                      <-Map 18 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_113]
+                                                          Group By Operator [GBY_112] (rows=1 width=8)
+                                                            Output:["_col0"],aggregations:["count()"]
+                                                            Select Operator [SEL_111] (rows=36524 width=1119)
+                                                              Filter Operator [FIL_261] (rows=36524 width=1119)
+                                                                predicate:(d_date = '1998-02-19')
+                                                                 Please refer to the previous TableScan [TS_9]
+                  <-Reducer 4 [SIMPLE_EDGE]
+                    SHUFFLE [RS_150]
                       PartitionCols:_col0
-                      Group By Operator [GBY_31] (rows=174233858 width=135)
+                      Group By Operator [GBY_48] (rows=174233858 width=135)
                         Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                      <-Reducer 4 [SIMPLE_EDGE]
-                        SHUFFLE [RS_30]
+                      <-Reducer 3 [SIMPLE_EDGE]
+                        SHUFFLE [RS_47]
                           PartitionCols:_col0
-                          Group By Operator [GBY_29] (rows=348467716 width=135)
-                            Output:["_col0","_col1"],aggregations:["sum(_col5)"],keys:_col7
-                            Merge Join Operator [MERGEJOIN_204] (rows=348467716 width=135)
-                              Conds:RS_25._col4=RS_26._col0(Inner),Output:["_col5","_col7"]
-                            <-Map 22 [SIMPLE_EDGE]
-                              SHUFFLE [RS_26]
+                          Group By Operator [GBY_46] (rows=348467716 width=135)
+                            Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col4
+                            Merge Join Operator [MERGEJOIN_276] (rows=348467716 width=135)
+                              Conds:RS_42._col0=RS_43._col0(Inner),Output:["_col2","_col4"]
+                            <-Reducer 15 [SIMPLE_EDGE]
+                              SHUFFLE [RS_43]
                                 PartitionCols:_col0
-                                Select Operator [SEL_18] (rows=462000 width=1436)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_190] (rows=462000 width=1436)
-                                    predicate:(i_item_sk is not null and i_item_id is not null)
-                                     Please refer to the previous TableScan [TS_16]
-                            <-Reducer 3 [SIMPLE_EDGE]
-                              SHUFFLE [RS_25]
-                                PartitionCols:_col4
-                                Merge Join Operator [MERGEJOIN_203] (rows=316788826 width=135)
-                                  Conds:RS_22._col0=RS_23._col0(Inner),Output:["_col4","_col5"]
-                                <-Map 21 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_23]
+                                Merge Join Operator [MERGEJOIN_267] (rows=80353 width=1119)
+                                  Conds:RS_35._col1=RS_36._col0(Inner),Output:["_col0"]
+                                <-Map 14 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_35]
+                                    PartitionCols:_col1
+                                    Select Operator [SEL_8] (rows=73049 width=1119)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_248] (rows=73049 width=1119)
+                                        predicate:(d_date is not null and d_date_sk is not null)
+                                         Please refer to the previous TableScan [TS_6]
+                                <-Reducer 22 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_36]
                                     PartitionCols:_col0
-                                    Select Operator [SEL_15] (rows=287989836 width=135)
+                                    Group By Operator [GBY_33] (rows=40176 width=1119)
+                                      Output:["_col0"],keys:KEY._col0
+                                    <-Reducer 21 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_32]
+                                        PartitionCols:_col0
+                                        Group By Operator [GBY_31] (rows=80353 width=1119)
+                                          Output:["_col0"],keys:_col2
+                                          Merge Join Operator [MERGEJOIN_266] (rows=80353 width=1119)
+                                            Conds:RS_27._col1=RS_28._col1(Inner),Output:["_col2"]
+                                          <-Map 31 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_28]
+                                              PartitionCols:_col1
+                                              Select Operator [SEL_23] (rows=73049 width=1119)
+                                                Output:["_col0","_col1"]
+                                                Filter Operator [FIL_251] (rows=73049 width=1119)
+                                                  predicate:(d_week_seq is not null and d_date is not null)
+                                                   Please refer to the previous TableScan [TS_18]
+                                          <-Reducer 20 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_27]
+                                              PartitionCols:_col1
+                                              Merge Join Operator [MERGEJOIN_265] (rows=36524 width=1128)
+                                                Conds:(Inner),Output:["_col1"]
+                                              <-Map 31 [CUSTOM_SIMPLE_EDGE]
+                                                SHUFFLE [RS_25]
+                                                  Select Operator [SEL_20] (rows=36524 width=1119)
+                                                    Output:["_col0"]
+                                                    Filter Operator [FIL_250] (rows=36524 width=1119)
+                                                      predicate:((d_date = '1998-02-19') and d_week_seq is not null)
+                                                       Please refer to the previous TableScan [TS_18]
+                                              <-Reducer 19 [CUSTOM_SIMPLE_EDGE]
+                                                PARTITION_ONLY_SHUFFLE [RS_24]
+                                                  Select Operator [SEL_17] (rows=1 width=8)
+                                                    Filter Operator [FIL_16] (rows=1 width=8)
+                                                      predicate:(sq_count_check(_col0) <= 1)
+                                                      Group By Operator [GBY_14] (rows=1 width=8)
+                                                        Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                                                      <-Map 18 [CUSTOM_SIMPLE_EDGE]
+                                                        PARTITION_ONLY_SHUFFLE [RS_13]
+                                                          Group By Operator [GBY_12] (rows=1 width=8)
+                                                            Output:["_col0"],aggregations:["count()"]
+                                                            Select Operator [SEL_11] (rows=36524 width=1119)
+                                                              Filter Operator [FIL_249] (rows=36524 width=1119)
+                                                                predicate:(d_date = '1998-02-19')
+                                                                 Please refer to the previous TableScan [TS_9]
+                            <-Reducer 2 [SIMPLE_EDGE]
+                              SHUFFLE [RS_42]
+                                PartitionCols:_col0
+                                Merge Join Operator [MERGEJOIN_264] (rows=316788826 width=135)
+                                  Conds:RS_39._col1=RS_40._col0(Inner),Output:["_col0","_col2","_col4"]
+                                <-Map 7 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_40]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_5] (rows=462000 width=1436)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_247] (rows=462000 width=1436)
+                                        predicate:(i_item_sk is not null and i_item_id is not null)
+                                         Please refer to the previous TableScan [TS_3]
+                                <-Map 1 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_39]
+                                    PartitionCols:_col1
+                                    Select Operator [SEL_2] (rows=287989836 width=135)
                                       Output:["_col0","_col1","_col2"]
-                                      Filter Operator [FIL_189] (rows=287989836 width=135)
+                                      Filter Operator [FIL_246] (rows=287989836 width=135)
                                         predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
-                                        TableScan [TS_13] (rows=287989836 width=135)
+                                        TableScan [TS_0] (rows=287989836 width=135)
                                           default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_ext_sales_price"]
-                                <-Reducer 2 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_22]
-                                    PartitionCols:_col0
-                                    Merge Join Operator [MERGEJOIN_202] (rows=88388 width=1119)
-                                      Conds:RS_19._col1=RS_20._col0(Inner),Output:["_col0"]
-                                    <-Map 1 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_19]
-                                        PartitionCols:_col1
-                                        Select Operator [SEL_2] (rows=73049 width=1119)
-                                          Output:["_col0","_col1"]
-                                          Filter Operator [FIL_186] (rows=73049 width=1119)
-                                            predicate:(d_date_sk is not null and d_date is not null)
-                                             Please refer to the previous TableScan [TS_0]
-                                    <-Reducer 17 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_20]
-                                        PartitionCols:_col0
-                                        Merge Join Operator [MERGEJOIN_201] (rows=80353 width=1119)
-                                          Conds:RS_9._col1=RS_10._col1(Inner),Output:["_col0"]
-                                        <-Map 16 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_9]
-                                            PartitionCols:_col1
-                                            Select Operator [SEL_5] (rows=73049 width=1119)
-                                              Output:["_col0","_col1"]
-                                              Filter Operator [FIL_187] (rows=73049 width=1119)
-                                                predicate:(d_week_seq is not null and d_date is not null)
-                                                 Please refer to the previous TableScan [TS_3]
-                                        <-Map 20 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_10]
-                                            PartitionCols:_col1
-                                            Select Operator [SEL_8] (rows=36524 width=1119)
-                                              Output:["_col1"]
-                                              Filter Operator [FIL_188] (rows=36524 width=1119)
-                                                predicate:((d_date = '1998-08-04') and d_week_seq is not null)
-                                                 Please refer to the previous TableScan [TS_6]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query59.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query59.q.out b/ql/src/test/results/clientpositive/perf/query59.q.out
new file mode 100644
index 0000000..b570b96
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query59.q.out
@@ -0,0 +1,238 @@
+PREHOOK: query: explain
+with wss as 
+ (select d_week_seq,
+        ss_store_sk,
+        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from store_sales,date_dim
+ where d_date_sk = ss_sold_date_sk
+ group by d_week_seq,ss_store_sk
+ )
+  select  s_store_name1,s_store_id1,d_week_seq1
+       ,sun_sales1/sun_sales2,mon_sales1/mon_sales2
+       ,tue_sales1/tue_sales1,wed_sales1/wed_sales2,thu_sales1/thu_sales2
+       ,fri_sales1/fri_sales2,sat_sales1/sat_sales2
+ from
+ (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1
+        ,s_store_id s_store_id1,sun_sales sun_sales1
+        ,mon_sales mon_sales1,tue_sales tue_sales1
+        ,wed_sales wed_sales1,thu_sales thu_sales1
+        ,fri_sales fri_sales1,sat_sales sat_sales1
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and 
+        d_month_seq between 1185 and 1185 + 11) y,
+ (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2
+        ,s_store_id s_store_id2,sun_sales sun_sales2
+        ,mon_sales mon_sales2,tue_sales tue_sales2
+        ,wed_sales wed_sales2,thu_sales thu_sales2
+        ,fri_sales fri_sales2,sat_sales sat_sales2
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and 
+        d_month_seq between 1185+ 12 and 1185 + 23) x
+ where s_store_id1=s_store_id2
+   and d_week_seq1=d_week_seq2-52
+ order by s_store_name1,s_store_id1,d_week_seq1
+limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+with wss as 
+ (select d_week_seq,
+        ss_store_sk,
+        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from store_sales,date_dim
+ where d_date_sk = ss_sold_date_sk
+ group by d_week_seq,ss_store_sk
+ )
+  select  s_store_name1,s_store_id1,d_week_seq1
+       ,sun_sales1/sun_sales2,mon_sales1/mon_sales2
+       ,tue_sales1/tue_sales1,wed_sales1/wed_sales2,thu_sales1/thu_sales2
+       ,fri_sales1/fri_sales2,sat_sales1/sat_sales2
+ from
+ (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1
+        ,s_store_id s_store_id1,sun_sales sun_sales1
+        ,mon_sales mon_sales1,tue_sales tue_sales1
+        ,wed_sales wed_sales1,thu_sales thu_sales1
+        ,fri_sales fri_sales1,sat_sales sat_sales1
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and 
+        d_month_seq between 1185 and 1185 + 11) y,
+ (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2
+        ,s_store_id s_store_id2,sun_sales sun_sales2
+        ,mon_sales mon_sales2,tue_sales tue_sales2
+        ,wed_sales wed_sales2,thu_sales thu_sales2
+        ,fri_sales fri_sales2,sat_sales sat_sales2
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and 
+        d_month_seq between 1185+ 12 and 1185 + 23) x
+ where s_store_id1=s_store_id2
+   and d_week_seq1=d_week_seq2-52
+ order by s_store_name1,s_store_id1,d_week_seq1
+limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 10 <- Map 13 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+Reducer 11 <- Map 15 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Map 14 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 11 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 7
+      File Output Operator [FS_63]
+        Limit [LIM_62] (rows=100 width=88)
+          Number of rows:100
+          Select Operator [SEL_61] (rows=421657640 width=88)
+            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
+          <-Reducer 6 [SIMPLE_EDGE]
+            SHUFFLE [RS_60]
+              Select Operator [SEL_59] (rows=421657640 width=88)
+                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
+                Merge Join Operator [MERGEJOIN_104] (rows=421657640 width=88)
+                  Conds:RS_56._col2, _col1=RS_57._col1, (_col0 - 52)(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col12","_col13","_col14","_col15","_col16","_col17"]
+                <-Reducer 11 [SIMPLE_EDGE]
+                  SHUFFLE [RS_57]
+                    PartitionCols:_col1, (_col0 - 52)
+                    Select Operator [SEL_55] (rows=383325119 width=88)
+                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+                      Merge Join Operator [MERGEJOIN_103] (rows=383325119 width=88)
+                        Conds:RS_52._col1=RS_53._col0(Inner),Output:["_col0","_col2","_col3","_col4","_col5","_col6","_col7","_col11"]
+                      <-Map 15 [SIMPLE_EDGE]
+                        SHUFFLE [RS_53]
+                          PartitionCols:_col0
+                          Select Operator [SEL_48] (rows=1704 width=1910)
+                            Output:["_col0","_col1"]
+                            Filter Operator [FIL_97] (rows=1704 width=1910)
+                              predicate:(s_store_sk is not null and s_store_id is not null)
+                              TableScan [TS_46] (rows=1704 width=1910)
+                                default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_store_id"]
+                      <-Reducer 10 [SIMPLE_EDGE]
+                        SHUFFLE [RS_52]
+                          PartitionCols:_col1
+                          Merge Join Operator [MERGEJOIN_102] (rows=348477374 width=88)
+                            Conds:RS_49._col0=RS_50._col1(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+                          <-Map 13 [SIMPLE_EDGE]
+                            SHUFFLE [RS_50]
+                              PartitionCols:_col1
+                              Select Operator [SEL_45] (rows=8116 width=1119)
+                                Output:["_col1"]
+                                Filter Operator [FIL_96] (rows=8116 width=1119)
+                                  predicate:(d_month_seq BETWEEN 1197 AND 1208 and d_week_seq is not null)
+                                  TableScan [TS_15] (rows=73049 width=1119)
+                                    default@date_dim,d,Tbl:COMPLETE,Col:NONE,Output:["d_month_seq","d_week_seq"]
+                          <-Reducer 9 [SIMPLE_EDGE]
+                            SHUFFLE [RS_49]
+                              PartitionCols:_col0
+                              Group By Operator [GBY_41] (rows=316797606 width=88)
+                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)"],keys:KEY._col0, KEY._col1
+                              <-Reducer 8 [SIMPLE_EDGE]
+                                SHUFFLE [RS_40]
+                                  PartitionCols:_col0, _col1
+                                  Group By Operator [GBY_39] (rows=633595212 width=88)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)","sum(_col3)","sum(_col5)","sum(_col6)","sum(_col7)","sum(_col8)"],keys:_col0, _col1
+                                    Select Operator [SEL_37] (rows=633595212 width=88)
+                                      Output:["_col0","_col1","_col2","_col3","_col5","_col6","_col7","_col8"]
+                                      Merge Join Operator [MERGEJOIN_101] (rows=633595212 width=88)
+                                        Conds:RS_34._col0=RS_35._col0(Inner),Output:["_col1","_col2","_col4","_col5"]
+                                      <-Map 1 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_34]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_30] (rows=575995635 width=88)
+                                            Output:["_col0","_col1","_col2"]
+                                            Filter Operator [FIL_94] (rows=575995635 width=88)
+                                              predicate:(ss_sold_date_sk is not null and ss_store_sk is not null)
+                                              TableScan [TS_0] (rows=575995635 width=88)
+                                                default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_store_sk","ss_sales_price"]
+                                      <-Map 12 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_35]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_33] (rows=73049 width=1119)
+                                            Output:["_col0","_col1","_col2"]
+                                            Filter Operator [FIL_95] (rows=73049 width=1119)
+                                              predicate:(d_date_sk is not null and d_week_seq is not null)
+                                              TableScan [TS_3] (rows=73049 width=1119)
+                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_week_seq","d_day_name"]
+                <-Reducer 5 [SIMPLE_EDGE]
+                  SHUFFLE [RS_56]
+                    PartitionCols:_col2, _col1
+                    Select Operator [SEL_27] (rows=383325119 width=88)
+                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
+                      Merge Join Operator [MERGEJOIN_100] (rows=383325119 width=88)
+                        Conds:RS_24._col1=RS_25._col0(Inner),Output:["_col0","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col12","_col13"]
+                      <-Map 14 [SIMPLE_EDGE]
+                        SHUFFLE [RS_25]
+                          PartitionCols:_col0
+                          Select Operator [SEL_20] (rows=1704 width=1910)
+                            Output:["_col0","_col1","_col2"]
+                            Filter Operator [FIL_93] (rows=1704 width=1910)
+                              predicate:(s_store_sk is not null and s_store_id is not null)
+                              TableScan [TS_18] (rows=1704 width=1910)
+                                default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_store_id","s_store_name"]
+                      <-Reducer 4 [SIMPLE_EDGE]
+                        SHUFFLE [RS_24]
+                          PartitionCols:_col1
+                          Merge Join Operator [MERGEJOIN_99] (rows=348477374 width=88)
+                            Conds:RS_21._col0=RS_22._col1(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
+                          <-Map 13 [SIMPLE_EDGE]
+                            SHUFFLE [RS_22]
+                              PartitionCols:_col1
+                              Select Operator [SEL_17] (rows=8116 width=1119)
+                                Output:["_col1"]
+                                Filter Operator [FIL_92] (rows=8116 width=1119)
+                                  predicate:(d_month_seq BETWEEN 1185 AND 1196 and d_week_seq is not null)
+                                   Please refer to the previous TableScan [TS_15]
+                          <-Reducer 3 [SIMPLE_EDGE]
+                            SHUFFLE [RS_21]
+                              PartitionCols:_col0
+                              Group By Operator [GBY_13] (rows=316797606 width=88)
+                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)"],keys:KEY._col0, KEY._col1
+                              <-Reducer 2 [SIMPLE_EDGE]
+                                SHUFFLE [RS_12]
+                                  PartitionCols:_col0, _col1
+                                  Group By Operator [GBY_11] (rows=633595212 width=88)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["sum(_col2)","sum(_col3)","sum(_col4)","sum(_col5)","sum(_col6)","sum(_col7)","sum(_col8)"],keys:_col0, _col1
+                                    Select Operator [SEL_9] (rows=633595212 width=88)
+                                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
+                                      Merge Join Operator [MERGEJOIN_98] (rows=633595212 width=88)
+                                        Conds:RS_6._col0=RS_7._col0(Inner),Output:["_col1","_col2","_col4","_col5"]
+                                      <-Map 1 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_6]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_2] (rows=575995635 width=88)
+                                            Output:["_col0","_col1","_col2"]
+                                            Filter Operator [FIL_90] (rows=575995635 width=88)
+                                              predicate:(ss_sold_date_sk is not null and ss_store_sk is not null)
+                                               Please refer to the previous TableScan [TS_0]
+                                      <-Map 12 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_7]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_5] (rows=73049 width=1119)
+                                            Output:["_col0","_col1","_col2"]
+                                            Filter Operator [FIL_91] (rows=73049 width=1119)
+                                              predicate:(d_date_sk is not null and d_week_seq is not null)
+                                               Please refer to the previous TableScan [TS_3]
+

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query6.q.out b/ql/src/test/results/clientpositive/perf/query6.q.out
index 04bda70..03fff79 100644
--- a/ql/src/test/results/clientpositive/perf/query6.q.out
+++ b/ql/src/test/results/clientpositive/perf/query6.q.out
@@ -1,5 +1,6 @@
 Warning: Shuffle Join MERGEJOIN[111][tables = [$hdt$_5, $hdt$_6]] in Stage 'Reducer 16' is a cross product
-PREHOOK: query: explain select  a.ca_state state, count(*) cnt
+PREHOOK: query: explain
+select  a.ca_state state, count(*) cnt
  from customer_address a
      ,customer c
      ,store_sales s
@@ -23,7 +24,8 @@ PREHOOK: query: explain select  a.ca_state state, count(*) cnt
  order by cnt 
  limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select  a.ca_state state, count(*) cnt
+POSTHOOK: query: explain
+select  a.ca_state state, count(*) cnt
  from customer_address a
      ,customer c
      ,store_sales s

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query60.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query60.q.out b/ql/src/test/results/clientpositive/perf/query60.q.out
index bb05a42..12d8cdd 100644
--- a/ql/src/test/results/clientpositive/perf/query60.q.out
+++ b/ql/src/test/results/clientpositive/perf/query60.q.out
@@ -1,4 +1,5 @@
-PREHOOK: query: explain with ss as (
+PREHOOK: query: explain
+with ss as (
  select
           i_item_id,sum(ss_ext_sales_price) total_sales
  from
@@ -74,7 +75,8 @@ where i_category in ('Children'))
       ,total_sales
  limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain with ss as (
+POSTHOOK: query: explain
+with ss as (
  select
           i_item_id,sum(ss_ext_sales_price) total_sales
  from

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query61.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query61.q.out b/ql/src/test/results/clientpositive/perf/query61.q.out
new file mode 100644
index 0000000..6838332
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query61.q.out
@@ -0,0 +1,300 @@
+Warning: Shuffle Join MERGEJOIN[156][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
+PREHOOK: query: explain
+select  promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100
+from
+  (select sum(ss_ext_sales_price) promotions
+   from  store_sales
+        ,store
+        ,promotion
+        ,date_dim
+        ,customer
+        ,customer_address 
+        ,item
+   where ss_sold_date_sk = d_date_sk
+   and   ss_store_sk = s_store_sk
+   and   ss_promo_sk = p_promo_sk
+   and   ss_customer_sk= c_customer_sk
+   and   ca_address_sk = c_current_addr_sk
+   and   ss_item_sk = i_item_sk 
+   and   ca_gmt_offset = -7
+   and   i_category = 'Electronics'
+   and   (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y')
+   and   s_gmt_offset = -7
+   and   d_year = 1999
+   and   d_moy  = 11) promotional_sales,
+  (select sum(ss_ext_sales_price) total
+   from  store_sales
+        ,store
+        ,date_dim
+        ,customer
+        ,customer_address
+        ,item
+   where ss_sold_date_sk = d_date_sk
+   and   ss_store_sk = s_store_sk
+   and   ss_customer_sk= c_customer_sk
+   and   ca_address_sk = c_current_addr_sk
+   and   ss_item_sk = i_item_sk
+   and   ca_gmt_offset = -7
+   and   i_category = 'Electronics'
+   and   s_gmt_offset = -7
+   and   d_year = 1999
+   and   d_moy  = 11) all_sales
+order by promotions, total
+limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select  promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100
+from
+  (select sum(ss_ext_sales_price) promotions
+   from  store_sales
+        ,store
+        ,promotion
+        ,date_dim
+        ,customer
+        ,customer_address 
+        ,item
+   where ss_sold_date_sk = d_date_sk
+   and   ss_store_sk = s_store_sk
+   and   ss_promo_sk = p_promo_sk
+   and   ss_customer_sk= c_customer_sk
+   and   ca_address_sk = c_current_addr_sk
+   and   ss_item_sk = i_item_sk 
+   and   ca_gmt_offset = -7
+   and   i_category = 'Electronics'
+   and   (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y')
+   and   s_gmt_offset = -7
+   and   d_year = 1999
+   and   d_moy  = 11) promotional_sales,
+  (select sum(ss_ext_sales_price) total
+   from  store_sales
+        ,store
+        ,date_dim
+        ,customer
+        ,customer_address
+        ,item
+   where ss_sold_date_sk = d_date_sk
+   and   ss_store_sk = s_store_sk
+   and   ss_customer_sk= c_customer_sk
+   and   ca_address_sk = c_current_addr_sk
+   and   ss_item_sk = i_item_sk
+   and   ca_gmt_offset = -7
+   and   i_category = 'Electronics'
+   and   s_gmt_offset = -7
+   and   d_year = 1999
+   and   d_moy  = 11) all_sales
+order by promotions, total
+limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 12 <- Map 11 (SIMPLE_EDGE), Map 16 (SIMPLE_EDGE)
+Reducer 13 <- Map 20 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE)
+Reducer 14 <- Map 21 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE)
+Reducer 15 <- Map 22 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE)
+Reducer 17 <- Map 16 (SIMPLE_EDGE), Map 23 (SIMPLE_EDGE)
+Reducer 18 <- Map 20 (SIMPLE_EDGE), Reducer 17 (SIMPLE_EDGE)
+Reducer 19 <- Map 21 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 15 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE), Reducer 9 (CUSTOM_SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
+Reducer 8 <- Reducer 19 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 8 (CUSTOM_SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:100
+    Stage-1
+      Reducer 6
+      File Output Operator [FS_91]
+        Limit [LIM_90] (rows=1 width=225)
+          Number of rows:100
+          Select Operator [SEL_89] (rows=1 width=225)
+            Output:["_col0","_col1","_col2"]
+          <-Reducer 5 [SIMPLE_EDGE]
+            SHUFFLE [RS_88]
+              Select Operator [SEL_87] (rows=1 width=225)
+                Output:["_col0","_col1","_col2"]
+                Merge Join Operator [MERGEJOIN_156] (rows=1 width=225)
+                  Conds:(Inner),Output:["_col0","_col1"]
+                <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_84]
+                    Group By Operator [GBY_43] (rows=1 width=112)
+                      Output:["_col0"],aggregations:["sum(VALUE._col0)"]
+                    <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
+                      PARTITION_ONLY_SHUFFLE [RS_42]
+                        Group By Operator [GBY_41] (rows=1 width=112)
+                          Output:["_col0"],aggregations:["sum(_col9)"]
+                          Merge Join Operator [MERGEJOIN_154] (rows=927646829 width=88)
+                            Conds:RS_37._col0=RS_38._col2(Inner),Output:["_col9"]
+                          <-Reducer 15 [SIMPLE_EDGE]
+                            SHUFFLE [RS_38]
+                              PartitionCols:_col2
+                              Merge Join Operator [MERGEJOIN_149] (rows=843315281 width=88)
+                                Conds:RS_30._col4=RS_31._col0(Inner),Output:["_col2","_col5"]
+                              <-Map 22 [SIMPLE_EDGE]
+                                SHUFFLE [RS_31]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_20] (rows=2300 width=1179)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_138] (rows=2300 width=1179)
+                                      predicate:(((p_channel_dmail = 'Y') or (p_channel_email = 'Y') or (p_channel_tv = 'Y')) and p_promo_sk is not null)
+                                      TableScan [TS_18] (rows=2300 width=1179)
+                                        default@promotion,promotion,Tbl:COMPLETE,Col:NONE,Output:["p_promo_sk","p_channel_dmail","p_channel_email","p_channel_tv"]
+                              <-Reducer 14 [SIMPLE_EDGE]
+                                SHUFFLE [RS_30]
+                                  PartitionCols:_col4
+                                  Merge Join Operator [MERGEJOIN_148] (rows=766650239 width=88)
+                                    Conds:RS_27._col3=RS_28._col0(Inner),Output:["_col2","_col4","_col5"]
+                                  <-Map 21 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_28]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_17] (rows=852 width=1910)
+                                        Output:["_col0"]
+                                        Filter Operator [FIL_137] (rows=852 width=1910)
+                                          predicate:((s_gmt_offset = -7) and s_store_sk is not null)
+                                          TableScan [TS_15] (rows=1704 width=1910)
+                                            default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_gmt_offset"]
+                                  <-Reducer 13 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_27]
+                                      PartitionCols:_col3
+                                      Merge Join Operator [MERGEJOIN_147] (rows=696954748 width=88)
+                                        Conds:RS_24._col1=RS_25._col0(Inner),Output:["_col2","_col3","_col4","_col5"]
+                                      <-Map 20 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_25]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_14] (rows=231000 width=1436)
+                                            Output:["_col0"]
+                                            Filter Operator [FIL_136] (rows=231000 width=1436)
+                                              predicate:((i_category = 'Electronics') and i_item_sk is not null)
+                                              TableScan [TS_12] (rows=462000 width=1436)
+                                                default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_category"]
+                                      <-Reducer 12 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_24]
+                                          PartitionCols:_col1
+                                          Merge Join Operator [MERGEJOIN_146] (rows=633595212 width=88)
+                                            Conds:RS_21._col0=RS_22._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
+                                          <-Map 16 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_22]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_11] (rows=18262 width=1119)
+                                                Output:["_col0"]
+                                                Filter Operator [FIL_135] (rows=18262 width=1119)
+                                                  predicate:((d_year = 1999) and (d_moy = 11) and d_date_sk is not null)
+                                                  TableScan [TS_9] (rows=73049 width=1119)
+                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
+                                          <-Map 11 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_21]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_8] (rows=575995635 width=88)
+                                                Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                                Filter Operator [FIL_134] (rows=575995635 width=88)
+                                                  predicate:(ss_store_sk is not null and ss_promo_sk is not null and ss_sold_date_sk is not null and ss_customer_sk is not null and ss_item_sk is not null)
+                                                  TableScan [TS_6] (rows=575995635 width=88)
+                                                    default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_store_sk","ss_promo_sk","ss_ext_sales_price"]
+                          <-Reducer 2 [SIMPLE_EDGE]
+                            SHUFFLE [RS_37]
+                              PartitionCols:_col0
+                              Merge Join Operator [MERGEJOIN_145] (rows=88000001 width=860)
+                                Conds:RS_34._col1=RS_35._col0(Inner),Output:["_col0"]
+                              <-Map 1 [SIMPLE_EDGE]
+                                SHUFFLE [RS_34]
+                                  PartitionCols:_col1
+                                  Select Operator [SEL_2] (rows=80000000 width=860)
+                                    Output:["_col0","_col1"]
+                                    Filter Operator [FIL_132] (rows=80000000 width=860)
+                                      predicate:(c_customer_sk is not null and c_current_addr_sk is not null)
+                                      TableScan [TS_0] (rows=80000000 width=860)
+                                        default@customer,customer,Tbl:COMPLETE,Col:NONE,Output:["c_customer_sk","c_current_addr_sk"]
+                              <-Map 10 [SIMPLE_EDGE]
+                                SHUFFLE [RS_35]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_5] (rows=20000000 width=1014)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_133] (rows=20000000 width=1014)
+                                      predicate:((ca_gmt_offset = -7) and ca_address_sk is not null)
+                                      TableScan [TS_3] (rows=40000000 width=1014)
+                                        default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_gmt_offset"]
+                <-Reducer 9 [CUSTOM_SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_85]
+                    Group By Operator [GBY_82] (rows=1 width=112)
+                      Output:["_col0"],aggregations:["sum(VALUE._col0)"]
+                    <-Reducer 8 [CUSTOM_SIMPLE_EDGE]
+                      PARTITION_ONLY_SHUFFLE [RS_81]
+                        Group By Operator [GBY_80] (rows=1 width=112)
+                          Output:["_col0"],aggregations:["sum(_col8)"]
+                          Merge Join Operator [MERGEJOIN_155] (rows=843315281 width=88)
+                            Conds:RS_76._col0=RS_77._col2(Inner),Output:["_col8"]
+                          <-Reducer 19 [SIMPLE_EDGE]
+                            SHUFFLE [RS_77]
+                              PartitionCols:_col2
+                              Merge Join Operator [MERGEJOIN_153] (rows=766650239 width=88)
+                                Conds:RS_69._col3=RS_70._col0(Inner),Output:["_col2","_col4"]
+                              <-Map 21 [SIMPLE_EDGE]
+                                SHUFFLE [RS_70]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_62] (rows=852 width=1910)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_144] (rows=852 width=1910)
+                                      predicate:((s_gmt_offset = -7) and s_store_sk is not null)
+                                       Please refer to the previous TableScan [TS_15]
+                              <-Reducer 18 [SIMPLE_EDGE]
+                                SHUFFLE [RS_69]
+                                  PartitionCols:_col3
+                                  Merge Join Operator [MERGEJOIN_152] (rows=696954748 width=88)
+                                    Conds:RS_66._col1=RS_67._col0(Inner),Output:["_col2","_col3","_col4"]
+                                  <-Map 20 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_67]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_59] (rows=231000 width=1436)
+                                        Output:["_col0"]
+                                        Filter Operator [FIL_143] (rows=231000 width=1436)
+                                          predicate:((i_category = 'Electronics') and i_item_sk is not null)
+                                           Please refer to the previous TableScan [TS_12]
+                                  <-Reducer 17 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_66]
+                                      PartitionCols:_col1
+                                      Merge Join Operator [MERGEJOIN_151] (rows=633595212 width=88)
+                                        Conds:RS_63._col0=RS_64._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                      <-Map 16 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_64]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_56] (rows=18262 width=1119)
+                                            Output:["_col0"]
+                                            Filter Operator [FIL_142] (rows=18262 width=1119)
+                                              predicate:((d_year = 1999) and (d_moy = 11) and d_date_sk is not null)
+                                               Please refer to the previous TableScan [TS_9]
+                                      <-Map 23 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_63]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_53] (rows=575995635 width=88)
+                                            Output:["_col0","_col1","_col2","_col3","_col4"]
+                                            Filter Operator [FIL_141] (rows=575995635 width=88)
+                                              predicate:(ss_store_sk is not null and ss_sold_date_sk is not null and ss_customer_sk is not null and ss_item_sk is not null)
+                                              TableScan [TS_51] (rows=575995635 width=88)
+                                                default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_store_sk","ss_ext_sales_price"]
+                          <-Reducer 7 [SIMPLE_EDGE]
+                            SHUFFLE [RS_76]
+                              PartitionCols:_col0
+                              Merge Join Operator [MERGEJOIN_150] (rows=88000001 width=860)
+                                Conds:RS_73._col1=RS_74._col0(Inner),Output:["_col0"]
+                              <-Map 1 [SIMPLE_EDGE]
+                                SHUFFLE [RS_73]
+                                  PartitionCols:_col1
+                                  Select Operator [SEL_47] (rows=80000000 width=860)
+                                    Output:["_col0","_col1"]
+                                    Filter Operator [FIL_139] (rows=80000000 width=860)
+                                      predicate:(c_customer_sk is not null and c_current_addr_sk is not null)
+                                       Please refer to the previous TableScan [TS_0]
+                              <-Map 10 [SIMPLE_EDGE]
+                                SHUFFLE [RS_74]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_50] (rows=20000000 width=1014)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_140] (rows=20000000 width=1014)
+                                      predicate:((ca_gmt_offset = -7) and ca_address_sk is not null)
+                                       Please refer to the previous TableScan [TS_3]
+


[02/17] hive git commit: HIVE-16764: Support numeric as same as decimal (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query95.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query95.q.out b/ql/src/test/results/clientpositive/perf/query95.q.out
index fa94d08..91d874e 100644
--- a/ql/src/test/results/clientpositive/perf/query95.q.out
+++ b/ql/src/test/results/clientpositive/perf/query95.q.out
@@ -1,154 +1,231 @@
-PREHOOK: query: explain SELECT count(distinct ws1.ws_order_number) as order_count, sum(ws1.ws_ext_ship_cost) as total_shipping_cost, sum(ws1.ws_net_profit) as total_net_profit FROM web_sales ws1 JOIN customer_address ca ON (ws1.ws_ship_addr_sk = ca.ca_address_sk) JOIN web_site s ON (ws1.ws_web_site_sk = s.web_site_sk) JOIN date_dim d ON (ws1.ws_ship_date_sk = d.d_date_sk) LEFT SEMI JOIN (SELECT ws2.ws_order_number as ws_order_number FROM web_sales ws2 JOIN web_sales ws3 ON (ws2.ws_order_number = ws3.ws_order_number) WHERE ws2.ws_warehouse_sk <> ws3.ws_warehouse_sk ) ws_wh1 ON (ws1.ws_order_number = ws_wh1.ws_order_number) LEFT SEMI JOIN (SELECT wr_order_number FROM web_returns wr JOIN (SELECT ws4.ws_order_number as ws_order_number FROM web_sales ws4 JOIN web_sales ws5 ON (ws4.ws_order_number = ws5.ws_order_number) WHERE ws4.ws_warehouse_sk <> ws5.ws_warehouse_sk ) ws_wh2 ON (wr.wr_order_number = ws_wh2.ws_order_number)) tmp1 ON (ws1.ws_order_number = tmp1.wr_order_number) WHERE d.d_
 date between '2002-05-01' and '2002-06-30' and ca.ca_state = 'GA' and s.web_company_name = 'pri'
+PREHOOK: query: explain
+with ws_wh as
+(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2
+ from web_sales ws1,web_sales ws2
+ where ws1.ws_order_number = ws2.ws_order_number
+   and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk)
+ select  
+   count(distinct ws_order_number) as `order count`
+  ,sum(ws_ext_ship_cost) as `total shipping cost`
+  ,sum(ws_net_profit) as `total net profit`
+from
+   web_sales ws1
+  ,date_dim
+  ,customer_address
+  ,web_site
+where
+    d_date between '1999-5-01' and 
+           (cast('1999-5-01' as date) + 60 days)
+and ws1.ws_ship_date_sk = d_date_sk
+and ws1.ws_ship_addr_sk = ca_address_sk
+and ca_state = 'TX'
+and ws1.ws_web_site_sk = web_site_sk
+and web_company_name = 'pri'
+and ws1.ws_order_number in (select ws_order_number
+                            from ws_wh)
+and ws1.ws_order_number in (select wr_order_number
+                            from web_returns,ws_wh
+                            where wr_order_number = ws_wh.ws_order_number)
+order by count(distinct ws_order_number)
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain SELECT count(distinct ws1.ws_order_number) as order_count, sum(ws1.ws_ext_ship_cost) as total_shipping_cost, sum(ws1.ws_net_profit) as total_net_profit FROM web_sales ws1 JOIN customer_address ca ON (ws1.ws_ship_addr_sk = ca.ca_address_sk) JOIN web_site s ON (ws1.ws_web_site_sk = s.web_site_sk) JOIN date_dim d ON (ws1.ws_ship_date_sk = d.d_date_sk) LEFT SEMI JOIN (SELECT ws2.ws_order_number as ws_order_number FROM web_sales ws2 JOIN web_sales ws3 ON (ws2.ws_order_number = ws3.ws_order_number) WHERE ws2.ws_warehouse_sk <> ws3.ws_warehouse_sk ) ws_wh1 ON (ws1.ws_order_number = ws_wh1.ws_order_number) LEFT SEMI JOIN (SELECT wr_order_number FROM web_returns wr JOIN (SELECT ws4.ws_order_number as ws_order_number FROM web_sales ws4 JOIN web_sales ws5 ON (ws4.ws_order_number = ws5.ws_order_number) WHERE ws4.ws_warehouse_sk <> ws5.ws_warehouse_sk ) ws_wh2 ON (wr.wr_order_number = ws_wh2.ws_order_number)) tmp1 ON (ws1.ws_order_number = tmp1.wr_order_number) WHERE d.d
 _date between '2002-05-01' and '2002-06-30' and ca.ca_state = 'GA' and s.web_company_name = 'pri'
+POSTHOOK: query: explain
+with ws_wh as
+(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2
+ from web_sales ws1,web_sales ws2
+ where ws1.ws_order_number = ws2.ws_order_number
+   and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk)
+ select  
+   count(distinct ws_order_number) as `order count`
+  ,sum(ws_ext_ship_cost) as `total shipping cost`
+  ,sum(ws_net_profit) as `total net profit`
+from
+   web_sales ws1
+  ,date_dim
+  ,customer_address
+  ,web_site
+where
+    d_date between '1999-5-01' and 
+           (cast('1999-5-01' as date) + 60 days)
+and ws1.ws_ship_date_sk = d_date_sk
+and ws1.ws_ship_addr_sk = ca_address_sk
+and ca_state = 'TX'
+and ws1.ws_web_site_sk = web_site_sk
+and web_company_name = 'pri'
+and ws1.ws_order_number in (select ws_order_number
+                            from ws_wh)
+and ws1.ws_order_number in (select wr_order_number
+                            from web_returns,ws_wh
+                            where wr_order_number = ws_wh.ws_order_number)
+order by count(distinct ws_order_number)
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 10 <- Map 12 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
-Reducer 3 <- Map 13 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 14 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 15 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 12 <- Map 11 (SIMPLE_EDGE), Map 17 (SIMPLE_EDGE)
+Reducer 13 <- Reducer 12 (SIMPLE_EDGE)
+Reducer 14 <- Map 11 (SIMPLE_EDGE), Map 17 (SIMPLE_EDGE)
+Reducer 15 <- Map 18 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE)
+Reducer 16 <- Reducer 15 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 3 <- Map 9 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 10 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 13 (SIMPLE_EDGE), Reducer 16 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
 Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
-Reducer 8 <- Map 11 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
-Reducer 9 <- Map 11 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 6
-      File Output Operator [FS_63]
-        Group By Operator [GBY_61] (rows=1 width=344)
-          Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT KEY._col0:0._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
-        <-Reducer 5 [SIMPLE_EDGE]
-          SHUFFLE [RS_60]
-            Group By Operator [GBY_59] (rows=510219083 width=135)
-              Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT _col3)","sum(_col4)","sum(_col5)"],keys:_col3
-              Merge Join Operator [MERGEJOIN_122] (rows=510219083 width=135)
-                Conds:RS_55._col2=RS_56._col0(Inner),Output:["_col3","_col4","_col5"]
-              <-Map 15 [SIMPLE_EDGE]
-                SHUFFLE [RS_56]
-                  PartitionCols:_col0
-                  Select Operator [SEL_40] (rows=42 width=1850)
-                    Output:["_col0"]
-                    Filter Operator [FIL_115] (rows=42 width=1850)
-                      predicate:((web_company_name = 'pri') and web_site_sk is not null)
-                      TableScan [TS_38] (rows=84 width=1850)
-                        default@web_site,s,Tbl:COMPLETE,Col:NONE,Output:["web_site_sk","web_company_name"]
-              <-Reducer 4 [SIMPLE_EDGE]
-                SHUFFLE [RS_55]
-                  PartitionCols:_col2
-                  Merge Join Operator [MERGEJOIN_121] (rows=463835520 width=135)
-                    Conds:RS_52._col1=RS_53._col0(Inner),Output:["_col2","_col3","_col4","_col5"]
-                  <-Map 14 [SIMPLE_EDGE]
-                    SHUFFLE [RS_53]
-                      PartitionCols:_col0
-                      Select Operator [SEL_37] (rows=20000000 width=1014)
-                        Output:["_col0"]
-                        Filter Operator [FIL_114] (rows=20000000 width=1014)
-                          predicate:((ca_state = 'GA') and ca_address_sk is not null)
-                          TableScan [TS_35] (rows=40000000 width=1014)
-                            default@customer_address,ca,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_state"]
-                  <-Reducer 3 [SIMPLE_EDGE]
-                    SHUFFLE [RS_52]
-                      PartitionCols:_col1
-                      Merge Join Operator [MERGEJOIN_120] (rows=421668646 width=135)
-                        Conds:RS_49._col0=RS_50._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
-                      <-Map 13 [SIMPLE_EDGE]
-                        SHUFFLE [RS_50]
+      Reducer 7
+      File Output Operator [FS_71]
+        Limit [LIM_69] (rows=1 width=344)
+          Number of rows:100
+          Select Operator [SEL_68] (rows=1 width=344)
+            Output:["_col0","_col1","_col2"]
+          <-Reducer 6 [SIMPLE_EDGE]
+            SHUFFLE [RS_67]
+              Select Operator [SEL_66] (rows=1 width=344)
+                Output:["_col1","_col2","_col3"]
+                Group By Operator [GBY_65] (rows=1 width=344)
+                  Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT KEY._col0:0._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
+                <-Reducer 5 [SIMPLE_EDGE]
+                  SHUFFLE [RS_64]
+                    Group By Operator [GBY_63] (rows=421668645 width=135)
+                      Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT _col3)","sum(_col4)","sum(_col5)"],keys:_col3
+                      Merge Join Operator [MERGEJOIN_121] (rows=421668645 width=135)
+                        Conds:RS_58._col3=RS_59._col0(Inner),RS_58._col3=RS_60._col0(Inner),Output:["_col3","_col4","_col5"]
+                      <-Reducer 13 [SIMPLE_EDGE]
+                        SHUFFLE [RS_59]
                           PartitionCols:_col0
-                          Select Operator [SEL_34] (rows=8116 width=1119)
-                            Output:["_col0"]
-                            Filter Operator [FIL_113] (rows=8116 width=1119)
-                              predicate:(d_date BETWEEN '2002-05-01' AND '2002-06-30' and d_date_sk is not null)
-                              TableScan [TS_32] (rows=73049 width=1119)
-                                default@date_dim,d,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
-                      <-Reducer 2 [SIMPLE_EDGE]
-                        SHUFFLE [RS_49]
-                          PartitionCols:_col0
-                          Merge Join Operator [MERGEJOIN_119] (rows=383335125 width=135)
-                            Conds:RS_45._col3=RS_46._col0(Left Semi),RS_45._col3=RS_47._col0(Left Semi),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                          <-Map 1 [SIMPLE_EDGE]
-                            SHUFFLE [RS_45]
-                              PartitionCols:_col3
-                              Select Operator [SEL_2] (rows=144002668 width=135)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                Filter Operator [FIL_107] (rows=144002668 width=135)
-                                  predicate:(ws_ship_addr_sk is not null and ws_web_site_sk is not null and ws_ship_date_sk is not null and ws_order_number is not null)
-                                  TableScan [TS_0] (rows=144002668 width=135)
-                                    default@web_sales,ws1,Tbl:COMPLETE,Col:NONE,Output:["ws_ship_date_sk","ws_ship_addr_sk","ws_web_site_sk","ws_order_number","ws_ext_ship_cost","ws_net_profit"]
-                          <-Reducer 10 [SIMPLE_EDGE]
-                            SHUFFLE [RS_47]
-                              PartitionCols:_col0
-                              Group By Operator [GBY_44] (rows=174243235 width=135)
-                                Output:["_col0"],keys:_col0
-                                Select Operator [SEL_31] (rows=174243235 width=135)
-                                  Output:["_col0"]
-                                  Merge Join Operator [MERGEJOIN_118] (rows=174243235 width=135)
-                                    Conds:RS_28._col0=RS_29._col0(Inner),Output:["_col1"]
-                                  <-Map 12 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_29]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_27] (rows=14398467 width=92)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_112] (rows=14398467 width=92)
-                                          predicate:wr_order_number is not null
-                                          TableScan [TS_25] (rows=14398467 width=92)
-                                            default@web_returns,wr,Tbl:COMPLETE,Col:NONE,Output:["wr_order_number"]
-                                  <-Reducer 9 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_28]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_24] (rows=158402938 width=135)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_23] (rows=158402938 width=135)
-                                          predicate:(_col0 <> _col2)
-                                          Merge Join Operator [MERGEJOIN_117] (rows=158402938 width=135)
-                                            Conds:RS_20._col1=RS_21._col1(Inner),Output:["_col0","_col1","_col2"]
-                                          <-Map 11 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_21]
-                                              PartitionCols:_col1
-                                              Select Operator [SEL_19] (rows=144002668 width=135)
-                                                Output:["_col0","_col1"]
-                                                Filter Operator [FIL_111] (rows=144002668 width=135)
-                                                  predicate:ws_order_number is not null
-                                                  TableScan [TS_6] (rows=144002668 width=135)
-                                                    default@web_sales,ws3,Tbl:COMPLETE,Col:NONE,Output:["ws_warehouse_sk","ws_order_number"]
-                                          <-Map 7 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_20]
-                                              PartitionCols:_col1
-                                              Select Operator [SEL_16] (rows=144002668 width=135)
-                                                Output:["_col0","_col1"]
-                                                Filter Operator [FIL_110] (rows=144002668 width=135)
-                                                  predicate:ws_order_number is not null
-                                                  TableScan [TS_3] (rows=144002668 width=135)
-                                                    default@web_sales,ws2,Tbl:COMPLETE,Col:NONE,Output:["ws_warehouse_sk","ws_order_number"]
-                          <-Reducer 8 [SIMPLE_EDGE]
-                            SHUFFLE [RS_46]
+                          Group By Operator [GBY_25] (rows=79201469 width=135)
+                            Output:["_col0"],keys:KEY._col0
+                          <-Reducer 12 [SIMPLE_EDGE]
+                            SHUFFLE [RS_24]
                               PartitionCols:_col0
-                              Group By Operator [GBY_42] (rows=158402938 width=135)
-                                Output:["_col0"],keys:_col0
-                                Select Operator [SEL_13] (rows=158402938 width=135)
-                                  Output:["_col0"]
-                                  Filter Operator [FIL_12] (rows=158402938 width=135)
+                              Group By Operator [GBY_23] (rows=158402938 width=135)
+                                Output:["_col0"],keys:_col1
+                                Select Operator [SEL_22] (rows=158402938 width=135)
+                                  Output:["_col1"]
+                                  Filter Operator [FIL_21] (rows=158402938 width=135)
                                     predicate:(_col0 <> _col2)
-                                    Merge Join Operator [MERGEJOIN_116] (rows=158402938 width=135)
-                                      Conds:RS_9._col1=RS_10._col1(Inner),Output:["_col0","_col1","_col2"]
+                                    Merge Join Operator [MERGEJOIN_118] (rows=158402938 width=135)
+                                      Conds:RS_18._col1=RS_19._col1(Inner),Output:["_col0","_col1","_col2"]
                                     <-Map 11 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_10]
+                                      SHUFFLE [RS_18]
                                         PartitionCols:_col1
-                                        Select Operator [SEL_8] (rows=144002668 width=135)
+                                        Select Operator [SEL_14] (rows=144002668 width=135)
                                           Output:["_col0","_col1"]
-                                          Filter Operator [FIL_109] (rows=144002668 width=135)
+                                          Filter Operator [FIL_110] (rows=144002668 width=135)
                                             predicate:ws_order_number is not null
-                                             Please refer to the previous TableScan [TS_6]
-                                    <-Map 7 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_9]
+                                            TableScan [TS_12] (rows=144002668 width=135)
+                                              default@web_sales,ws1,Tbl:COMPLETE,Col:NONE,Output:["ws_warehouse_sk","ws_order_number"]
+                                    <-Map 17 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_19]
                                         PartitionCols:_col1
-                                        Select Operator [SEL_5] (rows=144002668 width=135)
+                                        Select Operator [SEL_17] (rows=144002668 width=135)
                                           Output:["_col0","_col1"]
-                                          Filter Operator [FIL_108] (rows=144002668 width=135)
+                                          Filter Operator [FIL_111] (rows=144002668 width=135)
                                             predicate:ws_order_number is not null
-                                             Please refer to the previous TableScan [TS_3]
+                                            TableScan [TS_15] (rows=144002668 width=135)
+                                              default@web_sales,ws2,Tbl:COMPLETE,Col:NONE,Output:["ws_warehouse_sk","ws_order_number"]
+                      <-Reducer 16 [SIMPLE_EDGE]
+                        SHUFFLE [RS_60]
+                          PartitionCols:_col0
+                          Group By Operator [GBY_47] (rows=87121617 width=135)
+                            Output:["_col0"],keys:KEY._col0
+                          <-Reducer 15 [SIMPLE_EDGE]
+                            SHUFFLE [RS_46]
+                              PartitionCols:_col0
+                              Group By Operator [GBY_45] (rows=174243235 width=135)
+                                Output:["_col0"],keys:_col1
+                                Merge Join Operator [MERGEJOIN_120] (rows=174243235 width=135)
+                                  Conds:RS_41._col0=RS_42._col0(Inner),Output:["_col1"]
+                                <-Map 18 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_42]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_40] (rows=14398467 width=92)
+                                      Output:["_col0"]
+                                      Filter Operator [FIL_114] (rows=14398467 width=92)
+                                        predicate:wr_order_number is not null
+                                        TableScan [TS_38] (rows=14398467 width=92)
+                                          default@web_returns,web_returns,Tbl:COMPLETE,Col:NONE,Output:["wr_order_number"]
+                                <-Reducer 14 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_41]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_37] (rows=158402938 width=135)
+                                      Output:["_col0"]
+                                      Filter Operator [FIL_36] (rows=158402938 width=135)
+                                        predicate:(_col0 <> _col2)
+                                        Merge Join Operator [MERGEJOIN_119] (rows=158402938 width=135)
+                                          Conds:RS_33._col1=RS_34._col1(Inner),Output:["_col0","_col1","_col2"]
+                                        <-Map 11 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_33]
+                                            PartitionCols:_col1
+                                            Select Operator [SEL_29] (rows=144002668 width=135)
+                                              Output:["_col0","_col1"]
+                                              Filter Operator [FIL_112] (rows=144002668 width=135)
+                                                predicate:ws_order_number is not null
+                                                 Please refer to the previous TableScan [TS_12]
+                                        <-Map 17 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_34]
+                                            PartitionCols:_col1
+                                            Select Operator [SEL_32] (rows=144002668 width=135)
+                                              Output:["_col0","_col1"]
+                                              Filter Operator [FIL_113] (rows=144002668 width=135)
+                                                predicate:ws_order_number is not null
+                                                 Please refer to the previous TableScan [TS_15]
+                      <-Reducer 4 [SIMPLE_EDGE]
+                        SHUFFLE [RS_58]
+                          PartitionCols:_col3
+                          Merge Join Operator [MERGEJOIN_117] (rows=191667562 width=135)
+                            Conds:RS_55._col2=RS_56._col0(Inner),Output:["_col3","_col4","_col5"]
+                          <-Map 10 [SIMPLE_EDGE]
+                            SHUFFLE [RS_56]
+                              PartitionCols:_col0
+                              Select Operator [SEL_11] (rows=42 width=1850)
+                                Output:["_col0"]
+                                Filter Operator [FIL_109] (rows=42 width=1850)
+                                  predicate:((web_company_name = 'pri') and web_site_sk is not null)
+                                  TableScan [TS_9] (rows=84 width=1850)
+                                    default@web_site,web_site,Tbl:COMPLETE,Col:NONE,Output:["web_site_sk","web_company_name"]
+                          <-Reducer 3 [SIMPLE_EDGE]
+                            SHUFFLE [RS_55]
+                              PartitionCols:_col2
+                              Merge Join Operator [MERGEJOIN_116] (rows=174243235 width=135)
+                                Conds:RS_52._col1=RS_53._col0(Inner),Output:["_col2","_col3","_col4","_col5"]
+                              <-Map 9 [SIMPLE_EDGE]
+                                SHUFFLE [RS_53]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_8] (rows=20000000 width=1014)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_108] (rows=20000000 width=1014)
+                                      predicate:((ca_state = 'TX') and ca_address_sk is not null)
+                                      TableScan [TS_6] (rows=40000000 width=1014)
+                                        default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_state"]
+                              <-Reducer 2 [SIMPLE_EDGE]
+                                SHUFFLE [RS_52]
+                                  PartitionCols:_col1
+                                  Merge Join Operator [MERGEJOIN_115] (rows=158402938 width=135)
+                                    Conds:RS_49._col0=RS_50._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
+                                  <-Map 1 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_49]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_2] (rows=144002668 width=135)
+                                        Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                        Filter Operator [FIL_106] (rows=144002668 width=135)
+                                          predicate:(ws_order_number is not null and ws_ship_date_sk is not null and ws_ship_addr_sk is not null and ws_web_site_sk is not null)
+                                          TableScan [TS_0] (rows=144002668 width=135)
+                                            default@web_sales,ws1,Tbl:COMPLETE,Col:NONE,Output:["ws_ship_date_sk","ws_ship_addr_sk","ws_web_site_sk","ws_order_number","ws_ext_ship_cost","ws_net_profit"]
+                                  <-Map 8 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_50]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_5] (rows=8116 width=1119)
+                                        Output:["_col0"]
+                                        Filter Operator [FIL_107] (rows=8116 width=1119)
+                                          predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 1999-05-01 00:00:00.0 AND 1999-06-30 00:00:00.0 and d_date_sk is not null)
+                                          TableScan [TS_3] (rows=73049 width=1119)
+                                            default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query96.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query96.q.out b/ql/src/test/results/clientpositive/perf/query96.q.out
index a6c1f46..313fc7e 100644
--- a/ql/src/test/results/clientpositive/perf/query96.q.out
+++ b/ql/src/test/results/clientpositive/perf/query96.q.out
@@ -1,6 +1,32 @@
-PREHOOK: query: explain select count(*) as c from store_sales ,household_demographics ,time_dim, store where store_sales.ss_sold_time_sk = time_dim.t_time_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and store_sales.ss_store_sk = store.s_store_sk and time_dim.t_hour = 8 and time_dim.t_minute >= 30 and household_demographics.hd_dep_count = 5 and store.s_store_name = 'ese' order by c limit 100
+PREHOOK: query: explain
+select  count(*) 
+from store_sales
+    ,household_demographics 
+    ,time_dim, store
+where ss_sold_time_sk = time_dim.t_time_sk   
+    and ss_hdemo_sk = household_demographics.hd_demo_sk 
+    and ss_store_sk = s_store_sk
+    and time_dim.t_hour = 8
+    and time_dim.t_minute >= 30
+    and household_demographics.hd_dep_count = 5
+    and store.s_store_name = 'ese'
+order by count(*)
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select count(*) as c from store_sales ,household_demographics ,time_dim, store where store_sales.ss_sold_time_sk = time_dim.t_time_sk and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk and store_sales.ss_store_sk = store.s_store_sk and time_dim.t_hour = 8 and time_dim.t_minute >= 30 and household_demographics.hd_dep_count = 5 and store.s_store_name = 'ese' order by c limit 100
+POSTHOOK: query: explain
+select  count(*) 
+from store_sales
+    ,household_demographics 
+    ,time_dim, store
+where ss_sold_time_sk = time_dim.t_time_sk   
+    and ss_hdemo_sk = household_demographics.hd_demo_sk 
+    and ss_store_sk = s_store_sk
+    and time_dim.t_hour = 8
+    and time_dim.t_minute >= 30
+    and household_demographics.hd_dep_count = 5
+    and store.s_store_name = 'ese'
+order by count(*)
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -13,68 +39,70 @@ Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
       Reducer 6
-      File Output Operator [FS_29]
+      File Output Operator [FS_30]
         Limit [LIM_28] (rows=1 width=8)
           Number of rows:100
           Select Operator [SEL_27] (rows=1 width=8)
             Output:["_col0"]
           <-Reducer 5 [SIMPLE_EDGE]
             SHUFFLE [RS_26]
-              Group By Operator [GBY_24] (rows=1 width=8)
-                Output:["_col0"],aggregations:["count(VALUE._col0)"]
-              <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
-                PARTITION_ONLY_SHUFFLE [RS_23]
-                  Group By Operator [GBY_22] (rows=1 width=8)
-                    Output:["_col0"],aggregations:["count()"]
-                    Merge Join Operator [MERGEJOIN_45] (rows=766650239 width=88)
-                      Conds:RS_18._col2=RS_19._col0(Inner)
-                    <-Map 9 [SIMPLE_EDGE]
-                      SHUFFLE [RS_19]
-                        PartitionCols:_col0
-                        Select Operator [SEL_11] (rows=852 width=1910)
-                          Output:["_col0"]
-                          Filter Operator [FIL_42] (rows=852 width=1910)
-                            predicate:((s_store_name = 'ese') and s_store_sk is not null)
-                            TableScan [TS_9] (rows=1704 width=1910)
-                              default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_store_name"]
-                    <-Reducer 3 [SIMPLE_EDGE]
-                      SHUFFLE [RS_18]
-                        PartitionCols:_col2
-                        Merge Join Operator [MERGEJOIN_44] (rows=696954748 width=88)
-                          Conds:RS_15._col1=RS_16._col0(Inner),Output:["_col2"]
-                        <-Map 8 [SIMPLE_EDGE]
-                          SHUFFLE [RS_16]
-                            PartitionCols:_col0
-                            Select Operator [SEL_8] (rows=3600 width=107)
-                              Output:["_col0"]
-                              Filter Operator [FIL_41] (rows=3600 width=107)
-                                predicate:((hd_dep_count = 5) and hd_demo_sk is not null)
-                                TableScan [TS_6] (rows=7200 width=107)
-                                  default@household_demographics,household_demographics,Tbl:COMPLETE,Col:NONE,Output:["hd_demo_sk","hd_dep_count"]
-                        <-Reducer 2 [SIMPLE_EDGE]
-                          SHUFFLE [RS_15]
-                            PartitionCols:_col1
-                            Merge Join Operator [MERGEJOIN_43] (rows=633595212 width=88)
-                              Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col1","_col2"]
-                            <-Map 1 [SIMPLE_EDGE]
-                              SHUFFLE [RS_12]
-                                PartitionCols:_col0
-                                Select Operator [SEL_2] (rows=575995635 width=88)
-                                  Output:["_col0","_col1","_col2"]
-                                  Filter Operator [FIL_39] (rows=575995635 width=88)
-                                    predicate:(ss_hdemo_sk is not null and ss_sold_time_sk is not null and ss_store_sk is not null)
-                                    TableScan [TS_0] (rows=575995635 width=88)
-                                      default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_time_sk","ss_hdemo_sk","ss_store_sk"]
-                            <-Map 7 [SIMPLE_EDGE]
-                              SHUFFLE [RS_13]
-                                PartitionCols:_col0
-                                Select Operator [SEL_5] (rows=14400 width=471)
-                                  Output:["_col0"]
-                                  Filter Operator [FIL_40] (rows=14400 width=471)
-                                    predicate:((t_hour = 8) and (t_minute >= 30) and t_time_sk is not null)
-                                    TableScan [TS_3] (rows=86400 width=471)
-                                      default@time_dim,time_dim,Tbl:COMPLETE,Col:NONE,Output:["t_time_sk","t_hour","t_minute"]
+              Select Operator [SEL_25] (rows=1 width=8)
+                Output:["_col1"]
+                Group By Operator [GBY_24] (rows=1 width=8)
+                  Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_23]
+                    Group By Operator [GBY_22] (rows=1 width=8)
+                      Output:["_col0"],aggregations:["count()"]
+                      Merge Join Operator [MERGEJOIN_46] (rows=766650239 width=88)
+                        Conds:RS_18._col2=RS_19._col0(Inner)
+                      <-Map 9 [SIMPLE_EDGE]
+                        SHUFFLE [RS_19]
+                          PartitionCols:_col0
+                          Select Operator [SEL_11] (rows=852 width=1910)
+                            Output:["_col0"]
+                            Filter Operator [FIL_43] (rows=852 width=1910)
+                              predicate:((s_store_name = 'ese') and s_store_sk is not null)
+                              TableScan [TS_9] (rows=1704 width=1910)
+                                default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_store_name"]
+                      <-Reducer 3 [SIMPLE_EDGE]
+                        SHUFFLE [RS_18]
+                          PartitionCols:_col2
+                          Merge Join Operator [MERGEJOIN_45] (rows=696954748 width=88)
+                            Conds:RS_15._col1=RS_16._col0(Inner),Output:["_col2"]
+                          <-Map 8 [SIMPLE_EDGE]
+                            SHUFFLE [RS_16]
+                              PartitionCols:_col0
+                              Select Operator [SEL_8] (rows=3600 width=107)
+                                Output:["_col0"]
+                                Filter Operator [FIL_42] (rows=3600 width=107)
+                                  predicate:((hd_dep_count = 5) and hd_demo_sk is not null)
+                                  TableScan [TS_6] (rows=7200 width=107)
+                                    default@household_demographics,household_demographics,Tbl:COMPLETE,Col:NONE,Output:["hd_demo_sk","hd_dep_count"]
+                          <-Reducer 2 [SIMPLE_EDGE]
+                            SHUFFLE [RS_15]
+                              PartitionCols:_col1
+                              Merge Join Operator [MERGEJOIN_44] (rows=633595212 width=88)
+                                Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col1","_col2"]
+                              <-Map 1 [SIMPLE_EDGE]
+                                SHUFFLE [RS_12]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_2] (rows=575995635 width=88)
+                                    Output:["_col0","_col1","_col2"]
+                                    Filter Operator [FIL_40] (rows=575995635 width=88)
+                                      predicate:(ss_hdemo_sk is not null and ss_sold_time_sk is not null and ss_store_sk is not null)
+                                      TableScan [TS_0] (rows=575995635 width=88)
+                                        default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_time_sk","ss_hdemo_sk","ss_store_sk"]
+                              <-Map 7 [SIMPLE_EDGE]
+                                SHUFFLE [RS_13]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_5] (rows=14400 width=471)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_41] (rows=14400 width=471)
+                                      predicate:((t_hour = 8) and (t_minute >= 30) and t_time_sk is not null)
+                                      TableScan [TS_3] (rows=86400 width=471)
+                                        default@time_dim,time_dim,Tbl:COMPLETE,Col:NONE,Output:["t_time_sk","t_hour","t_minute"]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query97.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query97.q.out b/ql/src/test/results/clientpositive/perf/query97.q.out
index ef9a791..2d5129a 100644
--- a/ql/src/test/results/clientpositive/perf/query97.q.out
+++ b/ql/src/test/results/clientpositive/perf/query97.q.out
@@ -1,6 +1,50 @@
-PREHOOK: query: explain select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog from ( select ss_customer_sk customer_sk ,ss_item_sk item_sk from store_sales JOIN date_dim ON store_sales.ss_sold_date_sk = date_dim.d_date_sk where d_month_seq between 1193 and 1193 + 11 group by ss_customer_sk ,ss_item_sk) ssci full outer join ( select cs_bill_customer_sk customer_sk ,cs_item_sk item_sk from catalog_sales JOIN date_dim ON catalog_sales.cs_sold_date_sk = date_dim.d_date_sk where d_month_seq between 1193 and 1193 + 11 group by cs_bill_customer_sk ,cs_item_sk) csci on (ssci.customer_sk=csci.customer_sk and ssci.item_sk = csci.item_sk) limit 100
+PREHOOK: query: explain
+with ssci as (
+select ss_customer_sk customer_sk
+      ,ss_item_sk item_sk
+from store_sales,date_dim
+where ss_sold_date_sk = d_date_sk
+  and d_month_seq between 1212 and 1212 + 11
+group by ss_customer_sk
+        ,ss_item_sk),
+csci as(
+ select cs_bill_customer_sk customer_sk
+      ,cs_item_sk item_sk
+from catalog_sales,date_dim
+where cs_sold_date_sk = d_date_sk
+  and d_month_seq between 1212 and 1212 + 11
+group by cs_bill_customer_sk
+        ,cs_item_sk)
+ select  sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only
+      ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only
+      ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog
+from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk
+                               and ssci.item_sk = csci.item_sk)
+limit 100
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog from ( select ss_customer_sk customer_sk ,ss_item_sk item_sk from store_sales JOIN date_dim ON store_sales.ss_sold_date_sk = date_dim.d_date_sk where d_month_seq between 1193 and 1193 + 11 group by ss_customer_sk ,ss_item_sk) ssci full outer join ( select cs_bill_customer_sk customer_sk ,cs_item_sk item_sk from catalog_sales JOIN date_dim ON catalog_sales.cs_sold_date_sk = date_dim.d_date_sk where d_month_seq between 1193 and 1193 + 11 group by cs_bill_customer_sk ,cs_item_sk) csci on (ssci.customer_sk=csci.customer_sk and ssci.item_sk = csci.item_sk) limit 100
+POSTHOOK: query: explain
+with ssci as (
+select ss_customer_sk customer_sk
+      ,ss_item_sk item_sk
+from store_sales,date_dim
+where ss_sold_date_sk = d_date_sk
+  and d_month_seq between 1212 and 1212 + 11
+group by ss_customer_sk
+        ,ss_item_sk),
+csci as(
+ select cs_bill_customer_sk customer_sk
+      ,cs_item_sk item_sk
+from catalog_sales,date_dim
+where cs_sold_date_sk = d_date_sk
+  and d_month_seq between 1212 and 1212 + 11
+group by cs_bill_customer_sk
+        ,cs_item_sk)
+ select  sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only
+      ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only
+      ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog
+from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk
+                               and ssci.item_sk = csci.item_sk)
+limit 100
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -48,7 +92,7 @@ Stage-0
                                 Select Operator [SEL_5] (rows=8116 width=1119)
                                   Output:["_col0"]
                                   Filter Operator [FIL_44] (rows=8116 width=1119)
-                                    predicate:(d_month_seq BETWEEN 1193 AND 1204 and d_date_sk is not null)
+                                    predicate:(d_month_seq BETWEEN 1212 AND 1223 and d_date_sk is not null)
                                     TableScan [TS_3] (rows=73049 width=1119)
                                       default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_month_seq"]
                             <-Map 1 [SIMPLE_EDGE]
@@ -78,7 +122,7 @@ Stage-0
                                 Select Operator [SEL_19] (rows=8116 width=1119)
                                   Output:["_col0"]
                                   Filter Operator [FIL_46] (rows=8116 width=1119)
-                                    predicate:(d_month_seq BETWEEN 1193 AND 1204 and d_date_sk is not null)
+                                    predicate:(d_month_seq BETWEEN 1212 AND 1223 and d_date_sk is not null)
                                      Please refer to the previous TableScan [TS_3]
                             <-Map 9 [SIMPLE_EDGE]
                               SHUFFLE [RS_20]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query98.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query98.q.out b/ql/src/test/results/clientpositive/perf/query98.q.out
index 2894f7e..da534cd 100644
--- a/ql/src/test/results/clientpositive/perf/query98.q.out
+++ b/ql/src/test/results/clientpositive/perf/query98.q.out
@@ -1,16 +1,64 @@
-PREHOOK: query: explain 
-select i_item_desc ,i_category ,i_class ,i_current_price ,i_item_id ,sum(ss_ext_sales_price) as itemrevenue ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over (partition by i_class) as revenueratio from store_sales ,item ,date_dim 
-where store_sales.ss_item_sk = item.i_item_sk and i_category in ('Jewelry', 'Sports', 'Books') and store_sales.ss_sold_date_sk = date_dim.d_date_sk and 
-d_date between cast('2001-01-12' as date)
-                                and (cast('2001-01-12' as date) + 30 days)
-group by i_item_id ,i_item_desc ,i_category ,i_class ,i_current_price order by i_category ,i_class ,i_item_id ,i_item_desc ,revenueratio
+PREHOOK: query: explain
+select i_item_desc 
+      ,i_category 
+      ,i_class 
+      ,i_current_price
+      ,sum(ss_ext_sales_price) as itemrevenue 
+      ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over
+          (partition by i_class) as revenueratio
+from	
+	store_sales
+    	,item 
+    	,date_dim
+where 
+	ss_item_sk = i_item_sk 
+  	and i_category in ('Jewelry', 'Sports', 'Books')
+  	and ss_sold_date_sk = d_date_sk
+	and d_date between cast('2001-01-12' as date) 
+				and (cast('2001-01-12' as date) + 30 days)
+group by 
+	i_item_id
+        ,i_item_desc 
+        ,i_category
+        ,i_class
+        ,i_current_price
+order by 
+	i_category
+        ,i_class
+        ,i_item_id
+        ,i_item_desc
+        ,revenueratio
 PREHOOK: type: QUERY
-POSTHOOK: query: explain 
-select i_item_desc ,i_category ,i_class ,i_current_price ,i_item_id ,sum(ss_ext_sales_price) as itemrevenue ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over (partition by i_class) as revenueratio from store_sales ,item ,date_dim 
-where store_sales.ss_item_sk = item.i_item_sk and i_category in ('Jewelry', 'Sports', 'Books') and store_sales.ss_sold_date_sk = date_dim.d_date_sk and 
-d_date between cast('2001-01-12' as date)
-                                and (cast('2001-01-12' as date) + 30 days)
-group by i_item_id ,i_item_desc ,i_category ,i_class ,i_current_price order by i_category ,i_class ,i_item_id ,i_item_desc ,revenueratio
+POSTHOOK: query: explain
+select i_item_desc 
+      ,i_category 
+      ,i_class 
+      ,i_current_price
+      ,sum(ss_ext_sales_price) as itemrevenue 
+      ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over
+          (partition by i_class) as revenueratio
+from	
+	store_sales
+    	,item 
+    	,date_dim
+where 
+	ss_item_sk = i_item_sk 
+  	and i_category in ('Jewelry', 'Sports', 'Books')
+  	and ss_sold_date_sk = d_date_sk
+	and d_date between cast('2001-01-12' as date) 
+				and (cast('2001-01-12' as date) + 30 days)
+group by 
+	i_item_id
+        ,i_item_desc 
+        ,i_category
+        ,i_class
+        ,i_current_price
+order by 
+	i_category
+        ,i_class
+        ,i_item_id
+        ,i_item_desc
+        ,revenueratio
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
@@ -26,9 +74,9 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 6
-      File Output Operator [FS_27]
+      File Output Operator [FS_28]
         Select Operator [SEL_26] (rows=348477374 width=88)
-          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+          Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
         <-Reducer 5 [SIMPLE_EDGE]
           SHUFFLE [RS_25]
             Select Operator [SEL_23] (rows=348477374 width=88)
@@ -49,28 +97,28 @@ Stage-0
                           PartitionCols:_col0, _col1, _col2, _col3, _col4
                           Group By Operator [GBY_16] (rows=696954748 width=88)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col2)"],keys:_col10, _col9, _col6, _col7, _col8
-                            Merge Join Operator [MERGEJOIN_37] (rows=696954748 width=88)
+                            Merge Join Operator [MERGEJOIN_38] (rows=696954748 width=88)
                               Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col2","_col6","_col7","_col8","_col9","_col10"]
                             <-Map 8 [SIMPLE_EDGE]
                               SHUFFLE [RS_13]
                                 PartitionCols:_col0
                                 Select Operator [SEL_8] (rows=231000 width=1436)
                                   Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                  Filter Operator [FIL_35] (rows=231000 width=1436)
+                                  Filter Operator [FIL_36] (rows=231000 width=1436)
                                     predicate:((i_category) IN ('Jewelry', 'Sports', 'Books') and i_item_sk is not null)
                                     TableScan [TS_6] (rows=462000 width=1436)
                                       default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_item_id","i_item_desc","i_current_price","i_class","i_category"]
                             <-Reducer 2 [SIMPLE_EDGE]
                               SHUFFLE [RS_12]
                                 PartitionCols:_col1
-                                Merge Join Operator [MERGEJOIN_36] (rows=633595212 width=88)
+                                Merge Join Operator [MERGEJOIN_37] (rows=633595212 width=88)
                                   Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col1","_col2"]
                                 <-Map 1 [SIMPLE_EDGE]
                                   SHUFFLE [RS_9]
                                     PartitionCols:_col0
                                     Select Operator [SEL_2] (rows=575995635 width=88)
                                       Output:["_col0","_col1","_col2"]
-                                      Filter Operator [FIL_33] (rows=575995635 width=88)
+                                      Filter Operator [FIL_34] (rows=575995635 width=88)
                                         predicate:(ss_item_sk is not null and ss_sold_date_sk is not null)
                                         TableScan [TS_0] (rows=575995635 width=88)
                                           default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ext_sales_price"]
@@ -79,7 +127,7 @@ Stage-0
                                     PartitionCols:_col0
                                     Select Operator [SEL_5] (rows=8116 width=1119)
                                       Output:["_col0"]
-                                      Filter Operator [FIL_34] (rows=8116 width=1119)
+                                      Filter Operator [FIL_35] (rows=8116 width=1119)
                                         predicate:(CAST( d_date AS TIMESTAMP) BETWEEN 2001-01-12 00:00:00.0 AND 2001-02-11 00:00:00.0 and d_date_sk is not null)
                                         TableScan [TS_3] (rows=73049 width=1119)
                                           default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_date"]

http://git-wip-us.apache.org/repos/asf/hive/blob/86b18772/ql/src/test/results/clientpositive/perf/query99.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query99.q.out b/ql/src/test/results/clientpositive/perf/query99.q.out
new file mode 100644
index 0000000..e3e6bbe
--- /dev/null
+++ b/ql/src/test/results/clientpositive/perf/query99.q.out
@@ -0,0 +1,164 @@
+PREHOOK: query: explain
+select  
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,cc_name
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end)  as `30 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and 
+                 (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end )  as `31-60 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and 
+                 (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end)  as `61-90 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and
+                 (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end)  as `91-120 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk  > 120) then 1 else 0 end)  as `>120 days` 
+from
+   catalog_sales
+  ,warehouse
+  ,ship_mode
+  ,call_center
+  ,date_dim
+where
+    d_month_seq between 1212 and 1212 + 11
+and cs_ship_date_sk   = d_date_sk
+and cs_warehouse_sk   = w_warehouse_sk
+and cs_ship_mode_sk   = sm_ship_mode_sk
+and cs_call_center_sk = cc_call_center_sk
+group by
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,cc_name
+order by substr(w_warehouse_name,1,20)
+        ,sm_type
+        ,cc_name
+limit 100
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select  
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,cc_name
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end)  as `30 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and 
+                 (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end )  as `31-60 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and 
+                 (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end)  as `61-90 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and
+                 (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end)  as `91-120 days` 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk  > 120) then 1 else 0 end)  as `>120 days` 
+from
+   catalog_sales
+  ,warehouse
+  ,ship_mode
+  ,call_center
+  ,date_dim
+where
+    d_month_seq between 1212 and 1212 + 11
+and cs_ship_date_sk   = d_date_sk
+and cs_warehouse_sk   = w_warehouse_sk
+and cs_ship_mode_sk   = sm_ship_mode_sk
+and cs_call_center_sk = cc_call_center_sk
+group by
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,cc_name
+order by substr(w_warehouse_name,1,20)
+        ,sm_type
+        ,cc_name
+limit 100
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 3 <- Map 9 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 10 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Map 11 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 7
+      File Output Operator [FS_37]
+        Limit [LIM_35] (rows=100 width=135)
+          Number of rows:100
+          Select Operator [SEL_34] (rows=210822976 width=135)
+            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+          <-Reducer 6 [SIMPLE_EDGE]
+            SHUFFLE [RS_33]
+              Select Operator [SEL_32] (rows=210822976 width=135)
+                Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
+                Group By Operator [GBY_31] (rows=210822976 width=135)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)"],keys:KEY._col0, KEY._col1, KEY._col2
+                <-Reducer 5 [SIMPLE_EDGE]
+                  SHUFFLE [RS_30]
+                    PartitionCols:_col0, _col1, _col2
+                    Group By Operator [GBY_29] (rows=421645953 width=135)
+                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col3)","sum(_col4)","sum(_col5)","sum(_col6)","sum(_col7)"],keys:_col0, _col1, _col2
+                      Select Operator [SEL_27] (rows=421645953 width=135)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
+                        Merge Join Operator [MERGEJOIN_60] (rows=421645953 width=135)
+                          Conds:RS_24._col3=RS_25._col0(Inner),Output:["_col0","_col1","_col8","_col10","_col12"]
+                        <-Map 11 [SIMPLE_EDGE]
+                          SHUFFLE [RS_25]
+                            PartitionCols:_col0
+                            Select Operator [SEL_14] (rows=1 width=0)
+                              Output:["_col0","_col1"]
+                              Filter Operator [FIL_56] (rows=1 width=0)
+                                predicate: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_type"]
+                        <-Reducer 4 [SIMPLE_EDGE]
+                          SHUFFLE [RS_24]
+                            PartitionCols:_col3
+                            Merge Join Operator [MERGEJOIN_59] (rows=383314495 width=135)
+                              Conds:RS_21._col4=RS_22._col0(Inner),Output:["_col0","_col1","_col3","_col8","_col10"]
+                            <-Map 10 [SIMPLE_EDGE]
+                              SHUFFLE [RS_22]
+                                PartitionCols:_col0
+                                Select Operator [SEL_11] (rows=27 width=1029)
+                                  Output:["_col0","_col1"]
+                                  Filter Operator [FIL_55] (rows=27 width=1029)
+                                    predicate:w_warehouse_sk is not null
+                                    TableScan [TS_9] (rows=27 width=1029)
+                                      default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name"]
+                            <-Reducer 3 [SIMPLE_EDGE]
+                              SHUFFLE [RS_21]
+                                PartitionCols:_col4
+                                Merge Join Operator [MERGEJOIN_58] (rows=348467716 width=135)
+                                  Conds:RS_18._col2=RS_19._col0(Inner),Output:["_col0","_col1","_col3","_col4","_col8"]
+                                <-Map 9 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_19]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_8] (rows=60 width=2045)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_54] (rows=60 width=2045)
+                                        predicate:cc_call_center_sk is not null
+                                        TableScan [TS_6] (rows=60 width=2045)
+                                          default@call_center,call_center,Tbl:COMPLETE,Col:NONE,Output:["cc_call_center_sk","cc_name"]
+                                <-Reducer 2 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_18]
+                                    PartitionCols:_col2
+                                    Merge Join Operator [MERGEJOIN_57] (rows=316788826 width=135)
+                                      Conds:RS_15._col1=RS_16._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4"]
+                                    <-Map 1 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_15]
+                                        PartitionCols:_col1
+                                        Select Operator [SEL_2] (rows=287989836 width=135)
+                                          Output:["_col0","_col1","_col2","_col3","_col4"]
+                                          Filter Operator [FIL_52] (rows=287989836 width=135)
+                                            predicate:(cs_warehouse_sk is not null and cs_ship_mode_sk is not null and cs_call_center_sk is not null and cs_ship_date_sk is not null)
+                                            TableScan [TS_0] (rows=287989836 width=135)
+                                              default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:NONE,Output:["cs_sold_date_sk","cs_ship_date_sk","cs_call_center_sk","cs_ship_mode_sk","cs_warehouse_sk"]
+                                    <-Map 8 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_16]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_5] (rows=8116 width=1119)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_53] (rows=8116 width=1119)
+                                            predicate:(d_month_seq BETWEEN 1212 AND 1223 and d_date_sk is not null)
+                                            TableScan [TS_3] (rows=73049 width=1119)
+                                              default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_month_seq"]
+