You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by da...@apache.org on 2015/09/01 22:19:01 UTC

hive git commit: HIVE-11692: Fix UT regressions on hbase-metastore branch (Daniel Dai reviewed by Thejas Nair)

Repository: hive
Updated Branches:
  refs/heads/hbase-metastore 3d170cae5 -> 8b0ededf5


HIVE-11692: Fix UT regressions on hbase-metastore branch (Daniel Dai reviewed by Thejas Nair)


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

Branch: refs/heads/hbase-metastore
Commit: 8b0ededf574ff33c7fe4a952aad42ece1467237d
Parents: 3d170ca
Author: Daniel Dai <da...@hortonworks.com>
Authored: Tue Sep 1 13:18:35 2015 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Tue Sep 1 13:18:35 2015 -0700

----------------------------------------------------------------------
 data/conf/hbase/hive-site.xml                   | 263 -------------------
 .../hive/metastore/TestHiveMetaStore.java       |   3 +
 .../hive/metastore/hbase/TestHBaseImport.java   |  18 +-
 itests/qtest/pom.xml                            |  19 --
 .../test/resources/testconfiguration.properties |  46 ----
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   4 +
 metastore/pom.xml                               |   6 +
 .../hadoop/hive/metastore/TestObjectStore.java  |  43 ++-
 .../dynpart_sort_opt_vectorization.q.out        |  12 +-
 .../dynpart_sort_optimization.q.out             |  12 +-
 10 files changed, 76 insertions(+), 350 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/data/conf/hbase/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/hbase/hive-site.xml b/data/conf/hbase/hive-site.xml
deleted file mode 100644
index 2cde40f..0000000
--- a/data/conf/hbase/hive-site.xml
+++ /dev/null
@@ -1,263 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-
-<configuration>
-
-<property>
-  <name>hive.in.test</name>
-  <value>true</value>
-  <description>Internal marker for test. Used for masking env-dependent values</description>
-</property>
-
-<!-- Hive Configuration can either be stored in this file or in the hadoop configuration files  -->
-<!-- that are implied by Hadoop setup variables.                                                -->
-<!-- Aside from Hadoop setup variables - this file is provided as a convenience so that Hive    -->
-<!-- users do not have to edit hadoop configuration files (that may be managed as a centralized -->
-<!-- resource).                                                                                 -->
-
-<!-- Hive Execution Parameters -->
-<property>
-  <name>hadoop.tmp.dir</name>
-  <value>${test.tmp.dir}/hadoop-tmp</value>
-  <description>A base for other temporary directories.</description>
-</property>
-
-<!--
-<property>
-  <name>hive.exec.reducers.max</name>
-  <value>1</value>
-  <description>maximum number of reducers</description>
-</property>
--->
-
-<property>
-  <name>hive.exec.scratchdir</name>
-  <value>${test.tmp.dir}/scratchdir</value>
-  <description>Scratch space for Hive jobs</description>
-</property>
-
-<property>
-  <name>hive.exec.local.scratchdir</name>
-  <value>${test.tmp.dir}/localscratchdir/</value>
-  <description>Local scratch space for Hive jobs</description>
-</property>
-
-<property>
-  <name>javax.jdo.option.ConnectionURL</name>
-  <value>jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true</value>
-</property>
-
-<property>
-  <name>hive.stats.dbconnectionstring</name>
-  <value>jdbc:derby:;databaseName=${test.tmp.dir}/TempStatsStore;create=true</value>
-</property>
-
-
-<property>
-  <name>javax.jdo.option.ConnectionDriverName</name>
-  <value>org.apache.derby.jdbc.EmbeddedDriver</value>
-</property>
-
-<property>
-  <name>javax.jdo.option.ConnectionUserName</name>
-  <value>APP</value>
-</property>
-
-<property>
-  <name>javax.jdo.option.ConnectionPassword</name>
-  <value>mine</value>
-</property>
-
-<property>
-  <!--  this should eventually be deprecated since the metastore should supply this -->
-  <name>hive.metastore.warehouse.dir</name>
-  <value>${test.warehouse.dir}</value>
-  <description></description>
-</property>
-
-<property>
-  <name>hive.metastore.metadb.dir</name>
-  <value>file://${test.tmp.dir}/metadb/</value>
-  <description>
-  Required by metastore server or if the uris argument below is not supplied
-  </description>
-</property>
-
-<property>
-  <name>test.log.dir</name>
-  <value>${test.tmp.dir}/log/</value>
-  <description></description>
-</property>
-
-<property>
-  <name>test.data.files</name>
-  <value>${hive.root}/data/files</value>
-  <description></description>
-</property>
-
-<property>
-  <name>test.data.scripts</name>
-  <value>${hive.root}/data/scripts</value>
-  <description></description>
-</property>
-
-<property>
-  <name>hive.jar.path</name>
-  <value>${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar</value>
-  <description></description>
-</property>
-
-<property>
-  <name>hive.querylog.location</name>
-  <value>${test.tmp.dir}/tmp</value>
-  <description>Location of the structured hive logs</description>
-</property>
-
-<property>
-  <name>hive.exec.pre.hooks</name>
-  <value>org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables</value>
-  <description>Pre Execute Hook for Tests</description>
-</property>
-
-<property>
-  <name>hive.exec.post.hooks</name>
-  <value>org.apache.hadoop.hive.ql.hooks.PostExecutePrinter</value>
-  <description>Post Execute Hook for Tests</description>
-</property>
-
-<property>
-  <name>hive.support.concurrency</name>
-  <value>false</value>
-  <description>Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks.</description>
-</property>
-
-<property>
-  <key>hive.unlock.numretries</key>
-  <value>2</value>
-  <description>The number of times you want to retry to do one unlock</description>
-</property>
-
-<property>
-  <key>hive.lock.sleep.between.retries</key>
-  <value>2</value>
-  <description>The sleep time (in seconds) between various retries</description>
-</property>
-
-
-<property>
-  <name>fs.pfile.impl</name>
-  <value>org.apache.hadoop.fs.ProxyLocalFileSystem</value>
-  <description>A proxy for local file system used for cross file system testing</description>
-</property>
-
-<property>
-  <name>hive.exec.mode.local.auto</name>
-  <value>false</value>
-  <description>
-    Let hive determine whether to run in local mode automatically
-    Disabling this for tests so that minimr is not affected
-  </description>
-</property>
-
-<property>
-  <name>hive.auto.convert.join</name>
-  <value>false</value>
-  <description>Whether Hive enable the optimization about converting common join into mapjoin based on the input file size</description>
-</property>
-
-<property>
-  <name>hive.ignore.mapjoin.hint</name>
-  <value>false</value>
-  <description>Whether Hive ignores the mapjoin hint</description>
-</property>
-
-<property>
-  <name>hive.input.format</name>
-  <value>org.apache.hadoop.hive.ql.io.CombineHiveInputFormat</value>
-  <description>The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. </description>
-</property>
-
-<property>
-  <name>hive.default.rcfile.serde</name>
-  <value>org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe</value>
-  <description>The default SerDe hive will use for the rcfile format</description>
-</property>
-
-<property>
-  <name>hive.stats.dbclass</name>
-  <value>jdbc:derby</value>
-  <description>The storage for temporary stats generated by tasks. Currently, jdbc, hbase and counter types are supported</description>
-</property>
-
-<property>
-  <name>hive.stats.key.prefix.reserve.length</name>
-  <value>0</value>
-</property>
-
-<property>
-  <name>hive.conf.restricted.list</name>
-  <value>dummy.config.value</value>
-  <description>Using dummy config value above because you cannot override config with empty value</description>
-</property>
-
-<property>
-  <name>hive.exec.submit.local.task.via.child</name>
-  <value>false</value>
-</property>
-
-
-<property>
-  <name>hive.dummyparam.test.server.specific.config.override</name>
-  <value>from.hive-site.xml</value>
-  <description>Using dummy param to test server specific configuration</description>
-</property>
-
-<property>
-  <name>hive.dummyparam.test.server.specific.config.hivesite</name>
-  <value>from.hive-site.xml</value>
-  <description>Using dummy param to test server specific configuration</description>
-</property>
-
-<property>
-  <name>hive.ql.log.PerfLogger.level</name>
-  <value>WARN,DRFA</value>
-  <description>Used to change the perflogger level</description>
-</property>
-
-<property>
-  <name>hive.fetch.task.conversion</name>
-  <value>minimal</value>
-</property>
-
-<property>
-  <name>hive.users.in.admin.role</name>
-  <value>hive_admin_user</value>
-</property>
-
-<property>
-  <name>hive.metastore.fastpath</name>
-  <value>true</value>
-</property>
-
-<property>
-  <name>hive.metastore.rawstore.impl</name>
-  <value>org.apache.hadoop.hive.metastore.hbase.HBaseStore</value>
-</property>
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index 160667d..06061c0 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -2541,6 +2541,9 @@ public abstract class TestHiveMetaStore extends TestCase {
 
     try {
       cleanUp(dbName, null, null);
+      for (Function f : client.getAllFunctions().getFunctions()) {
+        client.dropFunction(f.getDbName(), f.getFunctionName());
+      }
 
       createDb(dbName);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
index 1ac10f0..2d2bd46 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseImport.java
@@ -22,6 +22,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.metastore.ObjectStore;
 import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.TestObjectStore;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Function;
@@ -57,22 +58,34 @@ import java.util.Set;
  */
 public class TestHBaseImport extends HBaseIntegrationTests {
 
-  private static final Log LOG = LogFactory.getLog(TestHBaseStoreIntegration.class.getName());
+  private static final Log LOG = LogFactory.getLog(TestHBaseImport.class.getName());
 
   private static final String[] tableNames = new String[] {"allnonparttable", "allparttable"};
   private static final String[] partVals = new String[] {"na", "emea", "latam", "apac"};
   private static final String[] funcNames = new String[] {"allfunc1", "allfunc2"};
 
+  private static final List<Integer> masterKeySeqs = new ArrayList<Integer>();
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @BeforeClass
   public static void startup() throws Exception {
     HBaseIntegrationTests.startMiniCluster();
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+    TestObjectStore.dropAllStoreObjects(rdbms);
   }
 
   @AfterClass
   public static void shutdown() throws Exception {
+    RawStore rdbms;
+    rdbms = new ObjectStore();
+    rdbms.setConf(conf);
+    TestObjectStore.dropAllStoreObjects(rdbms);
+    for (int seq : masterKeySeqs) {
+      rdbms.removeMasterKey(seq);
+    }
     HBaseIntegrationTests.shutdownMiniCluster();
   }
 
@@ -316,7 +329,6 @@ public class TestHBaseImport extends HBaseIntegrationTests {
     Assert.assertEquals(baseNumToks, store.getAllTokenIdentifiers().size());
     String[] hbaseKeys = store.getMasterKeys();
     Assert.assertEquals(baseNumKeys, hbaseKeys.length);
-
   }
 
   @Test
@@ -502,7 +514,7 @@ public class TestHBaseImport extends HBaseIntegrationTests {
     }
     for (int i = 0; i < tokenIds.length; i++) rdbms.addToken(tokenIds[i], tokens[i]);
     for (int i = 0; i < masterKeys.length; i++) {
-      rdbms.addMasterKey(masterKeys[i]);
+      masterKeySeqs.add(rdbms.addMasterKey(masterKeys[i]));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index 664068b..122e3f6 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -533,25 +533,6 @@
                   <else>
                   </else>
                 </if>
-                <!-- HBase Metastore -->
-                <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
-                          outputDirectory="${project.build.directory}/generated-test-sources/java/org/apache/hadoop/hive/cli/"
-                          templatePath="${basedir}/${hive.path.to.root}/ql/src/test/templates/" template="TestCliDriver.vm"
-                          queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/clientpositive/"
-                          queryFile="${qfile}"
-                          excludeQueryFile="${minimr.query.files},${minitez.query.files},${encrypted.query.files}"
-                          includeQueryFile="${miniHbaseMetastore.query.files}"
-                          queryFileRegex="${qfile_regex}"
-                          clusterMode="${clustermode}"
-                          runDisabled="${run_disabled}"
-                          hiveConfDir="${basedir}/${hive.path.to.root}/data/conf/hbase"
-                          resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/" 
-                          className="TestMiniHBaseMetastoreCliDriver"
-                          logFile="${project.build.directory}/testminihbasemetastoreclidrivergen.log"
-                          logDirectory="${project.build.directory}/qfile-results/clientpositive/"
-                          hadoopVersion="${active.hadoop.version}"
-                          initScript="q_test_init.sql"
-                          cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- Negative Minimr -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index c877f85..bed621d 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -1184,52 +1184,6 @@ miniSparkOnYarn.query.files=auto_sortmerge_join_16.q,\
   truncate_column_buckets.q,\
   uber_reduce.q
 
-miniHbaseMetastore.query.files=join1.q,\
-join2.q,\
-mapjoin1.q,\
-add_part_multiple.q,\
-annotate_stats_join.q,\
-authorization_parts.q,\
-auto_join1.q,\
-bucket1.q,\
-compute_stats_string.q,\
-create_1.q,\
-groupby1.q,\
-groupby12.q,\
-having.q,\
-innerjoin.q,\
-input_part10.q,\
-input20.q,\
-join1.q,\
-join20.q,\
-leftsemijoin.q,\
-mapjoin1.q,\
-multi_insert_gby.q,\
-orc_create.q,\
-orc_merge1.q,\
-show_roles.q,\
-stats0.q,\
-statsfs.q,\
-temp_table.q,\
-union.q,\
-union10.q,\
-alter_partition_change_col,\
-alter1.q,\
-analyze_tbl_part.q,\
-authorization_1.q,\
-columnstats_part_coltype.q,\
-ctas.q,\
-database.q,\
-drop_partition_with_stats.q,\
-drop_table_with_stats.q,\
-inputddl8.q,\
-order2.q,\
-partition_date.q,\
-partition_multilevels.q,\
-show_partitions.q,\
-sort.q,\
-view.q
-
 spark.query.negative.files=groupby2_map_skew_multi_distinct.q,\
   groupby2_multi_distinct.q,\
   groupby3_map_skew_multi_distinct.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 9f112ad..70df41e 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -288,6 +288,10 @@ public class QTestUtil {
       conf.setBoolVar(ConfVars.HIVE_VECTORIZATION_ENABLED, true);
     }
 
+    // Plug verifying metastore in for testing.
+    conf.setVar(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL,
+      "org.apache.hadoop.hive.metastore.VerifyingObjectStore");
+
     if (mr != null) {
       assert dfs != null;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/metastore/pom.xml
----------------------------------------------------------------------
diff --git a/metastore/pom.xml b/metastore/pom.xml
index 255726c..2ab5cd3 100644
--- a/metastore/pom.xml
+++ b/metastore/pom.xml
@@ -135,6 +135,12 @@
       <groupId>co.cask.tephra</groupId>
       <artifactId>tephra-core</artifactId>
       <version>${tephra.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.ow2.asm</groupId>
+          <artifactId>asm-all</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>co.cask.tephra</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
index a4f9f6c..78a9ea0 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -69,22 +70,13 @@ public class TestObjectStore {
   }
 
   @Before
-  public void setUp() {
+  public void setUp() throws Exception {
     HiveConf conf = new HiveConf();
     conf.setVar(HiveConf.ConfVars.METASTORE_EXPRESSION_PROXY_CLASS, MockPartitionExpressionProxy.class.getName());
 
     objectStore = new ObjectStore();
     objectStore.setConf(conf);
-
-    Deadline.registerIfNot(100000);
-    try {
-      objectStore.dropDatabase(DB1);
-    } catch (Exception e) {
-    }
-    try {
-      objectStore.dropDatabase(DB2);
-    } catch (Exception e) {
-    }
+    dropAllStoreObjects(objectStore);
   }
 
   @After
@@ -227,4 +219,33 @@ public class TestObjectStore {
     objectStore.revokeRole(role1, USER1, PrincipalType.USER, false);
     objectStore.removeRole(ROLE1);
   }
+
+  public static void dropAllStoreObjects(RawStore store) throws MetaException, InvalidObjectException, InvalidInputException {
+    try {
+      Deadline.registerIfNot(100000);
+      List<Function> funcs = store.getAllFunctions();
+      for (Function func : funcs) {
+        store.dropFunction(func.getDbName(), func.getFunctionName());
+      }
+      List<String> dbs = store.getAllDatabases();
+      for (int i = 0; i < dbs.size(); i++) {
+        String db = dbs.get(i);
+        List<String> tbls = store.getAllTables(db);
+        for (String tbl : tbls) {
+          Deadline.startTimer("getPartition");
+          List<Partition> parts = store.getPartitions(db, tbl, 100);
+          for (Partition part : parts) {
+            store.dropPartition(db, tbl, part.getValues());
+          }
+          store.dropTable(db, tbl);
+        }
+        store.dropDatabase(db);
+      }
+      List<String> roles = store.listRoleNames();
+      for (String role : roles) {
+        store.removeRole(role);
+      }
+    } catch (NoSuchObjectException e) {
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
index 216a79c..1f6339a 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
@@ -1755,13 +1755,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2_orc
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2_orc
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2_orc
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2_orc
 POSTHOOK: Input: default@over1k_part2_orc@ds=foo/t=27
@@ -1900,13 +1902,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2_orc
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2_orc
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2_orc@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2_orc
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2_orc
 POSTHOOK: Input: default@over1k_part2_orc@ds=foo/t=27

http://git-wip-us.apache.org/repos/asf/hive/blob/8b0ededf/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
index 41049bd..ebf4461 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
@@ -1655,13 +1655,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2
 PREHOOK: Input: default@over1k_part2@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2
 POSTHOOK: Input: default@over1k_part2@ds=foo/t=27
@@ -1800,13 +1802,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: select * from over1k_part2
+PREHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@over1k_part2
 PREHOOK: Input: default@over1k_part2@ds=foo/t=27
 PREHOOK: Input: default@over1k_part2@ds=foo/t=__HIVE_DEFAULT_PARTITION__
 #### A masked pattern was here ####
-POSTHOOK: query: select * from over1k_part2
+POSTHOOK: query: -- SORT_BEFORE_DIFF
+select * from over1k_part2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over1k_part2
 POSTHOOK: Input: default@over1k_part2@ds=foo/t=27