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

hive git commit: HIVE-20647: HadoopVer was ignored in QTestUtil (Denys Kuzmenko, reviewed by Peter Vary, Zoltan Haindrich, and Janaki Lahorani)

Repository: hive
Updated Branches:
  refs/heads/master a56ccebd5 -> 1e048df35


HIVE-20647: HadoopVer was ignored in QTestUtil (Denys Kuzmenko, reviewed by Peter Vary, Zoltan Haindrich, and Janaki Lahorani)


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

Branch: refs/heads/master
Commit: 1e048df35af7e685c5612b77f7198266ab3cf485
Parents: a56cceb
Author: denys kuzmenko <dk...@cloudera.com>
Authored: Mon Oct 8 10:54:00 2018 -0700
Committer: Janaki Lahorani <ja...@apache.org>
Committed: Mon Oct 8 11:12:29 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/TestLocationQueries.java     |   6 +-
 .../apache/hadoop/hive/ql/TestMTQueries.java    |   2 +-
 .../hadoop/hive/accumulo/AccumuloQTestUtil.java |  16 +-
 .../hadoop/hive/accumulo/AccumuloTestSetup.java |  21 ++-
 .../control/AbstractCoreBlobstoreCliDriver.java |  17 +-
 .../hive/cli/control/CoreAccumuloCliDriver.java |  25 ++-
 .../cli/control/CoreBlobstoreCliDriver.java     |   2 +-
 .../control/CoreBlobstoreNegativeCliDriver.java |   2 +-
 .../hadoop/hive/cli/control/CoreCliDriver.java  |  23 +--
 .../hive/cli/control/CoreCompareCliDriver.java  |  20 +--
 .../hadoop/hive/cli/control/CoreDummy.java      |   4 +-
 .../hive/cli/control/CoreHBaseCliDriver.java    |   7 +-
 .../cli/control/CoreHBaseNegativeCliDriver.java |   6 +-
 .../hive/cli/control/CoreNegativeCliDriver.java |  24 +--
 .../hive/cli/control/CorePerfCliDriver.java     |  20 +--
 .../hadoop/hive/hbase/HBaseQTestUtil.java       |  11 +-
 .../hadoop/hive/hbase/HBaseTestSetup.java       |   1 +
 .../apache/hadoop/hive/ql/QTestArguments.java   |  55 +++---
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 176 ++-----------------
 .../hadoop/hive/ql/parse/CoreParseNegative.java |  25 ++-
 20 files changed, 145 insertions(+), 318 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java
index b7e563a..405d21c 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java
@@ -86,8 +86,7 @@ public class TestLocationQueries extends BaseTestQueries {
       return QTestProcessExecResult.create(failedCount, fileNames.toString());
     }
 
-    public CheckResults(String outDir, String logDir, MiniClusterType miniMr,
-        String hadoopVer, String locationSubdir)
+    public CheckResults(String outDir, String logDir, MiniClusterType miniMr, String locationSubdir)
       throws Exception
     {
       super(
@@ -96,7 +95,6 @@ public class TestLocationQueries extends BaseTestQueries {
             .withLogDir(logDir)
             .withClusterType(miniMr)
             .withConfDir(null)
-            .withHadoopVer(hadoopVer)
             .withInitScript("")
             .withCleanupScript("")
             .withLlapIo(false)
@@ -119,7 +117,7 @@ public class TestLocationQueries extends BaseTestQueries {
     QTestUtil[] qt = new QTestUtil[qfiles.length];
 
     for (int i = 0; i < qfiles.length; i++) {
-      qt[i] = new CheckResults(resDir, logDir, MiniClusterType.none, "0.20", "parta");
+      qt[i] = new CheckResults(resDir, logDir, MiniClusterType.none, "parta");
       qt[i].newSession();
       qt[i].addFile(qfiles[i], false);
       qt[i].clearTestSideEffects();

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
index 3d8eb83..62c037e 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
@@ -46,7 +46,7 @@ public class TestMTQueries extends BaseTestQueries {
       util.getConf().set("hive.stats.dbclass", "fs");
       util.getConf().set("hive.mapred.mode", "nonstrict");
       util.getConf().set("hive.stats.column.autogather", "false");
-      util.newSession(true);
+      util.newSession();
     }
     boolean success = QTestUtil.queryListRunnerMultiThreaded(qfiles, qts);
     if (!success) {

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/accumulo/AccumuloQTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/accumulo/AccumuloQTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/accumulo/AccumuloQTestUtil.java
index 060e0cd..01c3c0b 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/accumulo/AccumuloQTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/accumulo/AccumuloQTestUtil.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.accumulo;
 
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QTestArguments;
 import org.apache.hadoop.hive.ql.QTestUtil;
 
@@ -25,7 +24,6 @@ import org.apache.hadoop.hive.ql.QTestUtil;
  * AccumuloQTestUtil initializes Accumulo-specific test fixtures.
  */
 public class AccumuloQTestUtil extends QTestUtil {
-  AccumuloTestSetup setup = null;
 
   public AccumuloQTestUtil(String outDir, String logDir, MiniClusterType miniMr,
       AccumuloTestSetup setup, String initScript, String cleanupScript) throws Exception {
@@ -36,22 +34,10 @@ public class AccumuloQTestUtil extends QTestUtil {
           .withLogDir(logDir)
           .withClusterType(miniMr)
           .withConfDir(null)
-          .withHadoopVer("0.20")
           .withInitScript(initScript)
           .withCleanupScript(cleanupScript)
           .withLlapIo(false)
+          .withQTestSetup(setup)
           .build());
-
-    setup.setupWithHiveConf(conf);
-    this.setup = setup;
-    this.savedConf = new HiveConf(conf);
-  }
-
-  @Override
-  public void initConf() throws Exception {
-    if (setup != null) {
-      setup.updateConf(conf);
-    }
-    super.initConf();
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/accumulo/AccumuloTestSetup.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/accumulo/AccumuloTestSetup.java b/itests/util/src/main/java/org/apache/hadoop/hive/accumulo/AccumuloTestSetup.java
index 47cf7ac..c599814 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/accumulo/AccumuloTestSetup.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/accumulo/AccumuloTestSetup.java
@@ -37,20 +37,28 @@ import org.apache.accumulo.minicluster.MiniAccumuloConfig;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QTestUtil;
 
 /**
  * Start and stop an AccumuloMiniCluster for testing purposes
  */
-public class AccumuloTestSetup  {
+public class AccumuloTestSetup extends QTestUtil.QTestSetup {
+
   public static final String PASSWORD = "password";
   public static final String TABLE_NAME = "accumuloHiveTable";
 
-  protected MiniAccumuloCluster miniCluster;
+  private MiniAccumuloCluster miniCluster;
 
   public AccumuloTestSetup() {
   }
 
-  protected void setupWithHiveConf(HiveConf conf) throws Exception {
+  @Override
+  public void preTest(HiveConf conf) throws Exception {
+    super.preTest(conf);
+    setupWithHiveConf(conf);
+  }
+
+  private void setupWithHiveConf(HiveConf conf) throws Exception {
     if (null == miniCluster) {
       String testTmpDir = System.getProperty("test.tmp.dir");
       File tmpDir = new File(testTmpDir, "accumulo");
@@ -63,7 +71,6 @@ public class AccumuloTestSetup  {
       cfg.setNumTservers(1);
 
       miniCluster = new MiniAccumuloCluster(cfg);
-
       miniCluster.start();
 
       createAccumuloTable(miniCluster.getConnector("root", PASSWORD));
@@ -76,7 +83,7 @@ public class AccumuloTestSetup  {
    * Update hiveConf with the Accumulo specific parameters
    * @param conf The hiveconf to update
    */
-  public void updateConf(HiveConf conf) {
+  private void updateConf(HiveConf conf) {
     // Setup connection information
     conf.set(AccumuloConnectionParameters.USER_NAME, "root");
     conf.set(AccumuloConnectionParameters.USER_PASS, PASSWORD);
@@ -86,7 +93,7 @@ public class AccumuloTestSetup  {
     }
   }
 
-  protected void createAccumuloTable(Connector conn) throws TableExistsException,
+  private void createAccumuloTable(Connector conn) throws TableExistsException,
       TableNotFoundException, AccumuloException, AccumuloSecurityException {
     TableOperations tops = conn.tableOperations();
     if (tops.exists(TABLE_NAME)) {
@@ -131,10 +138,12 @@ public class AccumuloTestSetup  {
     }
   }
 
+  @Override
   public void tearDown() throws Exception {
     if (null != miniCluster) {
       miniCluster.stop();
       miniCluster = null;
     }
+    super.tearDown();
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
index 64f2919..aa2c7a7 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
@@ -58,16 +58,14 @@ public abstract class AbstractCoreBlobstoreCliDriver extends CliAdapter {
     String hiveConfDir = cliConfig.getHiveConfDir();
     String initScript = cliConfig.getInitScript();
     String cleanupScript = cliConfig.getCleanupScript();
-    try {
-      String hadoopVer = cliConfig.getHadoopVersion();
 
+    try {
       qt = new QTestUtil(
           QTestArguments.QTestArgumentsBuilder.instance()
             .withOutDir(cliConfig.getResultsDir())
             .withLogDir(cliConfig.getLogDir())
             .withClusterType(miniMR)
             .withConfDir(hiveConfDir)
-            .withHadoopVer(hadoopVer)
             .withInitScript(initScript)
             .withCleanupScript(cleanupScript)
             .withLlapIo(true)
@@ -82,11 +80,12 @@ public abstract class AbstractCoreBlobstoreCliDriver extends CliAdapter {
       qt.newSession();
       qt.cleanUp();
       qt.createSources();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
       System.err.flush();
-      throw new RuntimeException("Unexpected exception in static initialization",e);
+      throw new RuntimeException("Unexpected exception in static initialization", e);
     }
   }
 
@@ -95,6 +94,7 @@ public abstract class AbstractCoreBlobstoreCliDriver extends CliAdapter {
   public void setUp() {
     try {
       qt.newSession();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -109,6 +109,7 @@ public abstract class AbstractCoreBlobstoreCliDriver extends CliAdapter {
     try {
       qt.clearTestSideEffects();
       qt.clearPostTestEffects();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -119,7 +120,7 @@ public abstract class AbstractCoreBlobstoreCliDriver extends CliAdapter {
 
   @Override
   @AfterClass
-  public void shutdown() throws Exception {
+  public void shutdown() {
     try {
       qt.shutdown();
       if (System.getenv(QTestUtil.QTEST_LEAVE_FILES) == null) {
@@ -134,10 +135,10 @@ public abstract class AbstractCoreBlobstoreCliDriver extends CliAdapter {
     }
   }
 
-  static String debugHint = "\nSee ./itests/hive-blobstore/target/tmp/log/hive.log, "
+  private static String debugHint = "\nSee ./itests/hive-blobstore/target/tmp/log/hive.log, "
       + "or check ./itests/hive-blobstore/target/surefire-reports/ for specific test cases logs.";
 
-  protected void runTestHelper(String tname, String fname, String fpath, boolean expectSuccess) throws Exception {
+  protected void runTestHelper(String tname, String fname, String fpath, boolean expectSuccess) {
     long startTime = System.currentTimeMillis();
     qt.getConf().set(HCONF_TEST_BLOBSTORE_PATH_UNIQUE, testBlobstorePathUnique);
     try {
@@ -145,10 +146,12 @@ public abstract class AbstractCoreBlobstoreCliDriver extends CliAdapter {
 
       qt.addFile(fpath);
       qt.cliInit(new File(fpath));
+
       int ecode = qt.executeClient(fname);
       if ((ecode == 0) ^ expectSuccess) {
         qt.failed(ecode, fname, debugHint);
       }
+
       QTestProcessExecResult result = qt.checkCliDriverResults(fname);
       if (result.getReturnCode() != 0) {
         String message = Strings.isNullOrEmpty(result.getCapturedOutput()) ?

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreAccumuloCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreAccumuloCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreAccumuloCliDriver.java
index 0d64cfa..6bbcf3d 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreAccumuloCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreAccumuloCliDriver.java
@@ -34,7 +34,6 @@ import org.junit.BeforeClass;
 public class CoreAccumuloCliDriver extends CliAdapter {
 
   private AccumuloQTestUtil qt;
-  private static AccumuloTestSetup setup;
 
   public CoreAccumuloCliDriver(AbstractCliConfig cliConfig) {
     super(cliConfig);
@@ -43,35 +42,32 @@ public class CoreAccumuloCliDriver extends CliAdapter {
   @Override
   @BeforeClass
   public void beforeClass() {
-    setup = new AccumuloTestSetup();
-
     MiniClusterType miniMR = cliConfig.getClusterType();
     String initScript = cliConfig.getInitScript();
     String cleanupScript = cliConfig.getCleanupScript();
 
     try {
       qt = new AccumuloQTestUtil(cliConfig.getResultsDir(), cliConfig.getLogDir(), miniMR,
-          setup, initScript, cleanupScript);
+          new AccumuloTestSetup(), initScript, cleanupScript);
 
       // do a one time initialization
       qt.newSession();
       qt.cleanUp();
       qt.createSources();
+
     } catch (Exception e) {
-      throw new RuntimeException("Unexpected exception in setUp",e);
+      throw new RuntimeException("Unexpected exception in setUp", e);
     }
   }
 
   @Override
   @AfterClass
-  public void shutdown() throws Exception {
-    setup.tearDown();
-
+  public void shutdown() {
     try {
       qt.shutdown();
-    }
-    catch (Exception e) {
-      throw new RuntimeException("Unexpected exception in tearDown",e);
+
+    } catch (Exception e) {
+      throw new RuntimeException("Unexpected exception in tearDown", e);
     }
   }
 
@@ -80,6 +76,7 @@ public class CoreAccumuloCliDriver extends CliAdapter {
   public void setUp() {
     try {
       qt.newSession();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -94,6 +91,7 @@ public class CoreAccumuloCliDriver extends CliAdapter {
     try {
       qt.clearPostTestEffects();
       qt.clearTestSideEffects();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -101,15 +99,16 @@ public class CoreAccumuloCliDriver extends CliAdapter {
       fail("Unexpected exception in tearDown");
     }
   }
+
   @Override
-  public void runTest(String tname, String fname, String fpath) throws Exception {
+  public void runTest(String tname, String fname, String fpath) {
     long startTime = System.currentTimeMillis();
     try {
       System.err.println("Begin query: " + fname);
 
       qt.addFile(fpath);
-
       qt.cliInit(new File(fpath));
+
       int ecode = qt.executeClient(fname);
       if (ecode != 0) {
         qt.failed(ecode, fname, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBlobstoreCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBlobstoreCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBlobstoreCliDriver.java
index bdb15b3..cd7e46f 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBlobstoreCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBlobstoreCliDriver.java
@@ -23,7 +23,7 @@ public class CoreBlobstoreCliDriver extends AbstractCoreBlobstoreCliDriver {
   }
 
   @Override
-  public void runTest(String tname, String fname, String fpath) throws Exception {
+  public void runTest(String tname, String fname, String fpath) {
     super.runTestHelper(tname, fname, fpath, true);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBlobstoreNegativeCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBlobstoreNegativeCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBlobstoreNegativeCliDriver.java
index 801c44b..4302907 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBlobstoreNegativeCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBlobstoreNegativeCliDriver.java
@@ -23,7 +23,7 @@ public class CoreBlobstoreNegativeCliDriver extends AbstractCoreBlobstoreCliDriv
   }
 
   @Override
-  public void runTest(String tname, String fname, String fpath) throws Exception {
+  public void runTest(String tname, String fname, String fpath) {
     super.runTestHelper(tname, fname, fpath, false);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
index 8413444..7ed8388 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
@@ -53,14 +53,13 @@ public class CoreCliDriver extends CliAdapter {
     String message = "Starting " + CoreCliDriver.class.getName() + " run at " + System.currentTimeMillis();
     LOG.info(message);
     System.err.println(message);
-    final MiniClusterType miniMR =cliConfig.getClusterType();
-    final String hiveConfDir = cliConfig.getHiveConfDir();
-    final String initScript = cliConfig.getInitScript();
-    final String cleanupScript = cliConfig.getCleanupScript();
 
-    try {
-      final String hadoopVer = cliConfig.getHadoopVersion();
+    MiniClusterType miniMR =cliConfig.getClusterType();
+    String hiveConfDir = cliConfig.getHiveConfDir();
+    String initScript = cliConfig.getInitScript();
+    String cleanupScript = cliConfig.getCleanupScript();
 
+    try {
       qt = new ElapsedTimeLoggingWrapper<QTestUtil>() {
         @Override
         public QTestUtil invokeInternal() throws Exception {
@@ -70,7 +69,6 @@ public class CoreCliDriver extends CliAdapter {
                 .withLogDir(cliConfig.getLogDir())
                 .withClusterType(miniMR)
                 .withConfDir(hiveConfDir)
-                .withHadoopVer(hadoopVer)
                 .withInitScript(initScript)
                 .withCleanupScript(cleanupScript)
                 .withLlapIo(true)
@@ -101,7 +99,7 @@ public class CoreCliDriver extends CliAdapter {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
       System.err.flush();
-      throw new RuntimeException("Unexpected exception in static initialization",e);
+      throw new RuntimeException("Unexpected exception in static initialization", e);
     }
   }
 
@@ -116,6 +114,7 @@ public class CoreCliDriver extends CliAdapter {
           return null;
         }
       }.invoke("PerTestSetup done.", LOG, false);
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -136,6 +135,7 @@ public class CoreCliDriver extends CliAdapter {
           return null;
         }
       }.invoke("PerTestTearDown done.", LOG, false);
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -146,7 +146,7 @@ public class CoreCliDriver extends CliAdapter {
 
   @Override
   @AfterClass
-  public void shutdown() throws Exception {
+  public void shutdown() {
     try {
       new ElapsedTimeLoggingWrapper<Void>() {
         @Override
@@ -155,6 +155,7 @@ public class CoreCliDriver extends CliAdapter {
           return null;
         }
       }.invoke("Teardown done.", LOG, false);
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -169,7 +170,7 @@ public class CoreCliDriver extends CliAdapter {
           + "or ./itests/qtest/target/surefire-reports/ for specific test cases logs.";
 
   @Override
-  public void runTest(String testName, String fname, String fpath) throws Exception {
+  public void runTest(String testName, String fname, String fpath) {
     Stopwatch sw = Stopwatch.createStarted();
     boolean skipped = false;
     boolean failed = false;
@@ -179,11 +180,13 @@ public class CoreCliDriver extends CliAdapter {
 
       qt.addFile(fpath);
       qt.cliInit(new File(fpath));
+
       int ecode = qt.executeClient(fname);
       if (ecode != 0) {
         failed = true;
         qt.failed(ecode, fname, debugHint);
       }
+
       QTestProcessExecResult result = qt.checkCliDriverResults(fname);
       if (result.getReturnCode() != 0) {
         failed = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java
index 8ce4349..7a06768 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java
@@ -21,9 +21,7 @@ import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.hive.ql.QTestArguments;
 import org.apache.hadoop.hive.ql.QTestProcessExecResult;
@@ -38,28 +36,26 @@ import com.google.common.base.Strings;
 public class CoreCompareCliDriver extends CliAdapter{
 
   private static QTestUtil qt;
+
   public CoreCompareCliDriver(AbstractCliConfig testCliConfig) {
     super(testCliConfig);
   }
 
-
   @Override
   @BeforeClass
   public void beforeClass() {
-
     MiniClusterType miniMR = cliConfig.getClusterType();
     String hiveConfDir = cliConfig.getHiveConfDir();
     String initScript = cliConfig.getInitScript();
     String cleanupScript = cliConfig.getCleanupScript();
+
     try {
-      String hadoopVer = cliConfig.getHadoopVersion();
       qt = new QTestUtil(
           QTestArguments.QTestArgumentsBuilder.instance()
             .withOutDir(cliConfig.getResultsDir())
             .withLogDir(cliConfig.getLogDir())
             .withClusterType(miniMR)
             .withConfDir(hiveConfDir)
-            .withHadoopVer(hadoopVer)
             .withInitScript(initScript)
             .withCleanupScript(cleanupScript)
             .withLlapIo(false)
@@ -74,7 +70,7 @@ public class CoreCompareCliDriver extends CliAdapter{
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
       System.err.flush();
-      fail("Unexpected exception in static initialization");
+      throw new RuntimeException("Unexpected exception in static initialization", e);
     }
   }
 
@@ -83,6 +79,7 @@ public class CoreCompareCliDriver extends CliAdapter{
   public void setUp() {
     try {
       qt.clearTestSideEffects();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -96,6 +93,7 @@ public class CoreCompareCliDriver extends CliAdapter{
   public void tearDown() {
     try {
       qt.clearPostTestEffects();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -106,7 +104,7 @@ public class CoreCompareCliDriver extends CliAdapter{
 
   @Override
   @AfterClass
-  public void shutdown() throws Exception {
+  public void shutdown() {
     try {
       qt.shutdown();
     } catch (Exception e) {
@@ -117,13 +115,11 @@ public class CoreCompareCliDriver extends CliAdapter{
     }
   }
 
-  private Map<String, List<String>> versionFiles = new HashMap<>();
-
-  static String debugHint = "\nSee ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, "
+  private static String debugHint = "\nSee ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, "
      + "or check ./ql/target/surefire-reports or ./itests/qtest/target/surefire-reports/ for specific test cases logs.";
 
   @Override
-  public void runTest(String tname, String fname, String fpath) throws Exception {
+  public void runTest(String tname, String fname, String fpath) {
     final String queryDirectory = cliConfig.getQueryDirectory();
 
     long startTime = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreDummy.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreDummy.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreDummy.java
index 4d40ef9..a0d1433 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreDummy.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreDummy.java
@@ -42,11 +42,11 @@ public class CoreDummy extends CliAdapter {
   }
 
   @Override
-  public void shutdown() throws Exception {
+  public void shutdown() {
   }
 
   @Override
-  public void runTest(String name, String name2, String absolutePath) throws Exception {
+  public void runTest(String name, String name2, String absolutePath) {
     List<String> versionFiles = QTestUtil.getVersionFiles(cliConfig.getQueryDirectory(), name);
     if (versionFiles.size() < 2) {
       fail("Cannot run " + name2 + " with only " + versionFiles.size() + " versions");

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java
index 252e9f6..0d67768 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java
@@ -58,9 +58,8 @@ public class CoreHBaseCliDriver extends CliAdapter {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
       System.err.flush();
-      fail("Unexpected exception in static initialization: "+e.getMessage());
+      throw new RuntimeException("Unexpected exception in static initialization: ", e);
     }
-
   }
 
   @Override
@@ -91,7 +90,7 @@ public class CoreHBaseCliDriver extends CliAdapter {
 
   @Override
   @AfterClass
-  public void shutdown() throws Exception {
+  public void shutdown() {
     try {
       qt.shutdown();
     } catch (Exception e) {
@@ -103,7 +102,7 @@ public class CoreHBaseCliDriver extends CliAdapter {
   }
 
   @Override
-  public void runTest(String tname, String fname, String fpath) throws Exception {
+  public void runTest(String tname, String fname, String fpath) {
     long startTime = System.currentTimeMillis();
     try {
       System.err.println("Begin query: " + fname);

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java
index c009cec..af170a9 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java
@@ -53,7 +53,7 @@ public class CoreHBaseNegativeCliDriver extends CliAdapter {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
       System.err.flush();
-      fail("Unexpected exception in static initialization: "+e.getMessage());
+      throw new RuntimeException("Unexpected exception in static initialization: ", e);
     }
   }
 
@@ -86,7 +86,7 @@ public class CoreHBaseNegativeCliDriver extends CliAdapter {
 
   @Override
   @AfterClass
-  public void shutdown() throws Exception {
+  public void shutdown() {
     try {
       qt.shutdown();
     } catch (Exception e) {
@@ -98,7 +98,7 @@ public class CoreHBaseNegativeCliDriver extends CliAdapter {
   }
 
   @Override
-  public void runTest(String tname, String fname, String fpath) throws Exception {
+  public void runTest(String tname, String fname, String fpath) {
     long startTime = System.currentTimeMillis();
     try {
       System.err.println("Begin query: " + fname);

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java
index 0807da1..91c3bf8 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java
@@ -35,6 +35,7 @@ import com.google.common.base.Strings;
 public class CoreNegativeCliDriver extends CliAdapter{
 
   private QTestUtil qt;
+
   public CoreNegativeCliDriver(AbstractCliConfig testCliConfig) {
     super(testCliConfig);
   }
@@ -47,14 +48,12 @@ public class CoreNegativeCliDriver extends CliAdapter{
     String cleanupScript = cliConfig.getCleanupScript();
 
     try {
-      String hadoopVer = cliConfig.getHadoopVersion();
       qt = new QTestUtil(
           QTestArguments.QTestArgumentsBuilder.instance()
             .withOutDir(cliConfig.getResultsDir())
             .withLogDir(cliConfig.getLogDir())
             .withClusterType(miniMR)
             .withConfDir(hiveConfDir)
-            .withHadoopVer(hadoopVer)
             .withInitScript(initScript)
             .withCleanupScript(cleanupScript)
             .withLlapIo(false)
@@ -63,11 +62,12 @@ public class CoreNegativeCliDriver extends CliAdapter{
       qt.newSession();
       qt.cleanUp();
       qt.createSources();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
       System.err.flush();
-      fail("Unexpected exception in static initialization");
+      throw new RuntimeException("Unexpected exception in static initialization", e);
     }
   }
 
@@ -76,6 +76,7 @@ public class CoreNegativeCliDriver extends CliAdapter{
   public void setUp() {
     try {
       qt.newSession();
+
     } catch (Throwable e) {
       e.printStackTrace();
       System.err.flush();
@@ -89,6 +90,7 @@ public class CoreNegativeCliDriver extends CliAdapter{
     try {
       qt.clearTestSideEffects();
       qt.clearPostTestEffects();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -99,9 +101,10 @@ public class CoreNegativeCliDriver extends CliAdapter{
 
   @Override
   @AfterClass
-  public void shutdown() throws Exception {
+  public void shutdown() {
     try {
       qt.shutdown();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -110,17 +113,9 @@ public class CoreNegativeCliDriver extends CliAdapter{
     }
   }
 
-  /**
-   * Dummy last test. This is only meant to shutdown qt
-   */
-  public void testNegativeCliDriver_shutdown() {
-    System.err.println ("Cleaning up " + "$className");
-  }
-
-  static String debugHint = "\nSee ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, "
+  private static String debugHint = "\nSee ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, "
      + "or check ./ql/target/surefire-reports or ./itests/qtest/target/surefire-reports/ for specific test cases logs.";
 
-
   @Override
   public void runTest(String tname, String fname, String fpath) throws Exception {
     long startTime = System.currentTimeMillis();
@@ -128,9 +123,8 @@ public class CoreNegativeCliDriver extends CliAdapter{
       System.err.println("Begin query: " + fname);
 
       qt.addFile(fpath);
-
-
       qt.cliInit(new File(fpath));
+
       int ecode = qt.executeClient(fname);
       if (ecode == 0) {
         qt.failed(fname, debugHint);

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java
index badb4a5..cf3d815 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java
@@ -58,20 +58,19 @@ public class CorePerfCliDriver extends CliAdapter{
   public void beforeClass() {
     System.setProperty("datanucleus.schema.autoCreateAll", "true");
     System.setProperty("hive.metastore.schema.verification", "false");
+
     MiniClusterType miniMR = cliConfig.getClusterType();
     String hiveConfDir = cliConfig.getHiveConfDir();
     String initScript = cliConfig.getInitScript();
     String cleanupScript = cliConfig.getCleanupScript();
-    try {
-      String hadoopVer = cliConfig.getHadoopVersion();
 
+    try {
       qt = new QTestUtil(
           QTestArguments.QTestArgumentsBuilder.instance()
             .withOutDir(cliConfig.getResultsDir())
             .withLogDir(cliConfig.getLogDir())
             .withClusterType(miniMR)
             .withConfDir(hiveConfDir)
-            .withHadoopVer(hadoopVer)
             .withInitScript(initScript)
             .withCleanupScript(cleanupScript)
             .withLlapIo(false)
@@ -85,12 +84,12 @@ public class CorePerfCliDriver extends CliAdapter{
       // the 30TB TPCDS scale set. This way the optimizer will generate plans for a 30 TB set.
       MetaStoreDumpUtility.setupMetaStoreTableColumnStatsFor30TBTPCDSWorkload(qt.getConf(),
           System.getProperty(QTestUtil.TEST_TMP_DIR_PROPERTY));
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
       System.err.flush();
-      throw new RuntimeException("Unexpected exception in static initialization: " + e.getMessage(),
-          e);
+      throw new RuntimeException("Unexpected exception in static initialization: " + e.getMessage(), e);
     }
   }
 
@@ -104,11 +103,12 @@ public class CorePerfCliDriver extends CliAdapter{
   public void setUp() {
     try {
       qt.newSession();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
       System.err.flush();
-      fail("Unexpected exception");
+      fail("Unexpected exception in setUp");
     }
   }
 
@@ -117,6 +117,7 @@ public class CorePerfCliDriver extends CliAdapter{
   public void tearDown() {
     try {
       qt.clearPostTestEffects();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
@@ -125,13 +126,12 @@ public class CorePerfCliDriver extends CliAdapter{
     }
   }
 
-  static String debugHint =
+  private static String debugHint =
       "\nSee ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, "
           + "or check ./ql/target/surefire-reports or ./itests/qtest/target/surefire-reports/ for specific test cases logs.";
 
-
   @Override
-  public void runTest(String name, String fname, String fpath) throws Exception {
+  public void runTest(String name, String fname, String fpath) {
     long startTime = System.currentTimeMillis();
     try {
       System.err.println("Begin query: " + fname);
@@ -143,6 +143,7 @@ public class CorePerfCliDriver extends CliAdapter{
       if (ecode != 0) {
         qt.failed(ecode, fname, debugHint);
       }
+
       QTestProcessExecResult result = qt.checkCliDriverResults(fname);
       if (result.getReturnCode() != 0) {
         String message = Strings.isNullOrEmpty(result.getCapturedOutput()) ?
@@ -158,5 +159,4 @@ public class CorePerfCliDriver extends CliAdapter{
     assertTrue("Test passed", true);
   }
 
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseQTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseQTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseQTestUtil.java
index 6a6b100..e8e5176 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseQTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseQTestUtil.java
@@ -48,7 +48,6 @@ public class HBaseQTestUtil extends QTestUtil {
           .withLogDir(logDir)
           .withClusterType(miniMr)
           .withConfDir(null)
-          .withHadoopVer("0.20")
           .withInitScript(initScript)
           .withCleanupScript(cleanupScript)
           .withLlapIo(false)
@@ -63,7 +62,6 @@ public class HBaseQTestUtil extends QTestUtil {
     super.createSources(tname);
 
     conf.setBoolean("hive.test.init.phase", true);
-
     initDataset(HBASE_SRC_NAME);
 
     // create a snapshot
@@ -72,9 +70,10 @@ public class HBaseQTestUtil extends QTestUtil {
       admin = conn.getAdmin();
       admin.snapshot(HBASE_SRC_SNAPSHOT_NAME, TableName.valueOf(HBASE_SRC_NAME));
     } finally {
-      if (admin != null) admin.close();
+      if (admin != null) {
+        admin.close();
+      }
     }
-
     conf.setBoolean("hive.test.init.phase", false);
   }
 
@@ -87,7 +86,9 @@ public class HBaseQTestUtil extends QTestUtil {
       admin = conn.getAdmin();
       admin.deleteSnapshots(HBASE_SRC_SNAPSHOT_NAME);
     } finally {
-      if (admin != null) admin.close();
+      if (admin != null) {
+        admin.close();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseTestSetup.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseTestSetup.java b/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseTestSetup.java
index cc63a6c..ec38aea 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseTestSetup.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseTestSetup.java
@@ -94,6 +94,7 @@ public class HBaseTestSetup extends QTestUtil.QTestSetup {
 
     Configuration hbaseConf = HBaseConfiguration.create(conf);
     util = new HBaseTestingUtility(hbaseConf);
+
     util.startMiniDFSCluster(1);
     hbaseCluster = util.startMiniHBaseCluster(1, NUM_REGIONSERVERS);
     hbaseConn = util.getConnection();

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestArguments.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestArguments.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestArguments.java
index 18269eb..33412b2 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestArguments.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestArguments.java
@@ -21,32 +21,34 @@ package org.apache.hadoop.hive.ql;
 /**
  * QTestArguments composite used as arguments holder for QTestUtil initialization.
  */
-public class QTestArguments {
+public final class QTestArguments {
 
   private String outDir;
   private String logDir;
   private String confDir;
   private QTestUtil.MiniClusterType clusterType;
-  private String hadoopVer;
   private String initScript;
   private String cleanupScript;
   private boolean withLlapIo;
   private QTestUtil.FsType fsType;
   private QTestUtil.QTestSetup qtestSetup;
 
-  public void setClusterType(QTestUtil.MiniClusterType clusterType) {
-    this.clusterType = clusterType;
+  private QTestArguments() {
   }
 
   public QTestUtil.MiniClusterType getClusterType() {
     return clusterType;
   }
 
+  private void setClusterType(QTestUtil.MiniClusterType clusterType) {
+    this.clusterType = clusterType;
+  }
+
   public String getOutDir() {
     return outDir;
   }
 
-  public void setOutDir(String outDir) {
+  private void setOutDir(String outDir) {
     this.outDir = outDir;
   }
 
@@ -54,27 +56,19 @@ public class QTestArguments {
     return logDir;
   }
 
-  public void setLogDir(String logDir) {
+  private void setLogDir(String logDir) {
     this.logDir = logDir;
   }
 
-  public void setConfDir(String confDir) {
-    this.confDir = confDir;
-  }
-
   public String getConfDir() {
     return confDir;
   }
 
-  public void setHadoopVer(String hadoopVer) {
-    this.hadoopVer = hadoopVer;
-  }
-
-  public String getHadoopVer() {
-    return hadoopVer;
+  private void setConfDir(String confDir) {
+    this.confDir = confDir;
   }
 
-  public void setInitScript(String initScript) {
+  private void setInitScript(String initScript) {
     this.initScript = initScript;
   }
 
@@ -82,7 +76,7 @@ public class QTestArguments {
     return initScript;
   }
 
-  public void setCleanupScript(String cleanupScript) {
+  private void setCleanupScript(String cleanupScript) {
     this.cleanupScript = cleanupScript;
   }
 
@@ -90,39 +84,39 @@ public class QTestArguments {
     return cleanupScript;
   }
 
-  public void setWithLlapIo(boolean withLlapIo) {
-    this.withLlapIo = withLlapIo;
-  }
-
   public boolean isWithLlapIo() {
     return withLlapIo;
   }
 
-  public void setFsType(QTestUtil.FsType fsType) {
-    this.fsType = fsType;
+  private void setWithLlapIo(boolean withLlapIo) {
+    this.withLlapIo = withLlapIo;
   }
 
   public QTestUtil.FsType getFsType() {
     return fsType;
   }
 
-  public void setQTestSetup(QTestUtil.QTestSetup qtestSetup) {
-    this.qtestSetup = qtestSetup;
+  private void setFsType(QTestUtil.FsType fsType) {
+    this.fsType = fsType;
   }
 
   public QTestUtil.QTestSetup getQTestSetup() {
     return qtestSetup;
   }
 
+  private void setQTestSetup(QTestUtil.QTestSetup qtestSetup) {
+    this.qtestSetup = qtestSetup;
+  }
+
   /**
    * QTestArgumentsBuilder used for QTestArguments construction.
    */
   public static final class QTestArgumentsBuilder {
+
     private String outDir;
     private String logDir;
     private String confDir;
     private QTestUtil.MiniClusterType clusterType;
-    private String hadoopVer;
     private String initScript;
     private String cleanupScript;
     private boolean withLlapIo;
@@ -156,12 +150,6 @@ public class QTestArguments {
       return this;
     }
 
-
-    public QTestArgumentsBuilder withHadoopVer(String hadoopVer) {
-      this.hadoopVer = hadoopVer;
-      return this;
-    }
-
     public QTestArgumentsBuilder withInitScript(String initScript) {
       this.initScript = initScript;
       return this;
@@ -193,7 +181,6 @@ public class QTestArguments {
       testArguments.setLogDir(logDir);
       testArguments.setConfDir(confDir);
       testArguments.setClusterType(clusterType);
-      testArguments.setHadoopVer(hadoopVer);
       testArguments.setInitScript(initScript);
       testArguments.setCleanupScript(cleanupScript);
       testArguments.setWithLlapIo(withLlapIo);

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/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 0e8b829..5924d06 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
@@ -33,7 +33,6 @@ import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.PrintStream;
 import java.io.StringWriter;
-import java.io.UnsupportedEncodingException;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
@@ -84,7 +83,6 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.LlapItUtils;
 import org.apache.hadoop.hive.llap.daemon.MiniLlapCluster;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
-import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.ql.cache.results.QueryResultsCache;
 import org.apache.hadoop.hive.ql.dataset.Dataset;
@@ -135,22 +133,20 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 
-import junit.framework.TestSuite;
-
 /**
  * QTestUtil.
  *
  */
 public class QTestUtil {
 
-  public static final String UTF_8 = "UTF-8";
+  private static final Logger LOG = LoggerFactory.getLogger("QTestUtil");
+  private static final String UTF_8 = "UTF-8";
 
   // security property names
   private static final String SECURITY_KEY_PROVIDER_URI_NAME = "dfs.encryption.key.provider.uri";
   private static final String CRLF = System.getProperty("line.separator");
 
   public static final String QTEST_LEAVE_FILES = "QTEST_LEAVE_FILES";
-  static final Logger LOG = LoggerFactory.getLogger("QTestUtil");
   private final static String defaultInitScript = "q_test_init.sql";
   private final static String defaultCleanupScript = "q_test_cleanup.sql";
   private static SimpleDateFormat formatter = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss");
@@ -158,9 +154,7 @@ public class QTestUtil {
 
   public static final String TEST_TMP_DIR_PROPERTY = "test.tmp.dir"; // typically target/tmp
   private static final String BUILD_DIR_PROPERTY = "build.dir"; // typically target
-
-  public static final String TEST_SRC_TABLES_PROPERTY = "test.src.tables";
-  public static final String TEST_HIVE_USER_PROPERTY = "test.hive.user";
+  private static final String TEST_SRC_TABLES_PROPERTY = "test.src.tables";
 
   /**
    * The default Erasure Coding Policy to use in Erasure Coding tests.
@@ -171,8 +165,7 @@ public class QTestUtil {
   @Deprecated
   private final String testFiles;
   private final File datasetDir;
-  protected final String outDir;
-  protected String overrideResultsDir;
+  private final String outDir;
   protected final String logDir;
   private final TreeMap<String, String> qMap;
   private final Set<String> qSortSet;
@@ -190,23 +183,21 @@ public class QTestUtil {
   private final FsType fsType;
   private ParseDriver pd;
   protected Hive db;
-  protected QueryState queryState;
+  private QueryState queryState;
   protected HiveConf conf;
   protected HiveConf savedConf;
   private IDriver drv;
   private BaseSemanticAnalyzer sem;
-  protected final boolean overWrite;
+  private final boolean overWrite;
   private CliDriver cliDriver;
   private HadoopShims.MiniMrShim mr = null;
   private HadoopShims.MiniDFSShim dfs = null;
   private FileSystem fs;
   private HadoopShims.HdfsEncryptionShim hes = null;
   private MiniLlapCluster llapCluster = null;
-  private String hadoopVer = null;
-  private QTestSetup setup = null;
+  private final QTestSetup setup;
   private SparkSession sparkSession = null;
   private boolean isSessionStateStarted = false;
-  private static final String javaVersion = getJavaVersion();
   private QOutProcessor qOutProcessor;
   private final String initScript;
   private final String cleanupScript;
@@ -214,10 +205,6 @@ public class QTestUtil {
   private MiniDruidCluster druidCluster;
   private SingleNodeKafkaCluster kafkaCluster;
 
-  public interface SuiteAddTestFunctor {
-    public void addTestToSuite(TestSuite suite, Object setup, String tName);
-  }
-
   public static Set<String> getSrcTables() {
     if (srcTables == null){
       initSrcTables();
@@ -262,7 +249,6 @@ public class QTestUtil {
     return cliDriver;
   }
 
-
   /**
    * Returns the default UDF names which should not be removed when resetting the test database
    * @return The list of the UDF names not to remove
@@ -289,18 +275,6 @@ public class QTestUtil {
     return conf;
   }
 
-  private String getHadoopMainVersion(String input) {
-    if (input == null) {
-      return null;
-    }
-    Pattern p = Pattern.compile("^(\\d+\\.\\d+).*");
-    Matcher m = p.matcher(input);
-    if (m.matches()) {
-      return m.group(1);
-    }
-    return null;
-  }
-
   public void initConf() throws Exception {
 
     String vectorizationEnabled = System.getProperty("test.vectorization.enabled");
@@ -482,7 +456,6 @@ public class QTestUtil {
     }
   }
 
-
   private String getKeyProviderURI() {
     // Use the target directory if it is not specified
     String HIVE_ROOT = AbstractCliConfig.HIVE_ROOT;
@@ -494,12 +467,12 @@ public class QTestUtil {
 
   public QTestUtil(QTestArguments testArgs) throws Exception {
     LOG.info("Setting up QTestUtil with outDir={}, logDir={}, clusterType={}, confDir={}," +
-        " hadoopVer={}, initScript={}, cleanupScript={}, withLlapIo={}," +
-            " fsType={}",
-        testArgs.getOutDir(), testArgs.getLogDir(), testArgs.getClusterType(), testArgs.getConfDir(), hadoopVer,
-            testArgs.getInitScript(), testArgs.getCleanupScript(), testArgs.isWithLlapIo(), testArgs.getFsType());
+        " initScript={}, cleanupScript={}, withLlapIo={}, fsType={}",
+        testArgs.getOutDir(), testArgs.getLogDir(), testArgs.getClusterType(), testArgs.getConfDir(),
+        testArgs.getInitScript(), testArgs.getCleanupScript(), testArgs.isWithLlapIo(), testArgs.getFsType());
 
     Preconditions.checkNotNull(testArgs.getClusterType(), "ClusterType cannot be null");
+
     this.fsType = testArgs.getFsType();
     this.outDir = testArgs.getOutDir();
     this.logDir = testArgs.getLogDir();
@@ -516,7 +489,6 @@ public class QTestUtil {
 
     queryState = new QueryState.Builder().withHiveConf(new HiveConf(IDriver.class)).build();
     conf = queryState.getConf();
-    this.hadoopVer = getHadoopMainVersion(hadoopVer);
     qMap = new TreeMap<String, String>();
     qSortSet = new HashSet<String>();
     qSortQuerySet = new HashSet<String>();
@@ -529,7 +501,6 @@ public class QTestUtil {
     this.clusterType = testArgs.getClusterType();
 
     HadoopShims shims = ShimLoader.getHadoopShims();
-
     setupFileSystem(shims);
 
     setup = testArgs.getQTestSetup();
@@ -568,6 +539,7 @@ public class QTestUtil {
     init();
     savedConf = new HiveConf(conf);
   }
+
   private String getScriptsDir() {
     // Use the current directory if it is not specified
     String scriptsDir = conf.get("test.data.scripts");
@@ -682,7 +654,6 @@ public class QTestUtil {
           MiniClusterType.druidKafka
       ).contains(clusterType)) {
         llapCluster = LlapItUtils.startAndGetMiniLlapCluster(conf, setup.zooKeeperCluster, confDir);
-      } else {
       }
       if (EnumSet.of(MiniClusterType.llap_local, MiniClusterType.tez_local, MiniClusterType.druidLocal)
                  .contains(clusterType)) {
@@ -854,27 +825,6 @@ public class QTestUtil {
   }
 
   /**
-   * Get formatted Java version to include minor version, but
-   * exclude patch level.
-   *
-   * @return Java version formatted as major_version.minor_version
-   */
-  private static String getJavaVersion() {
-    String version = System.getProperty("java.version");
-    if (version == null) {
-      throw new NullPointerException("No java version could be determined " +
-          "from system properties");
-    }
-
-    // "java version" system property is formatted
-    // major_version.minor_version.patch_level.
-    // Find second dot, instead of last dot, to be safe
-    int pos = version.indexOf('.');
-    pos = version.indexOf('.', pos + 1);
-    return version.substring(0, pos);
-  }
-
-  /**
    * Clear out any side effects of running tests
    */
   public void clearPostTestEffects() throws Exception {
@@ -1001,13 +951,10 @@ public class QTestUtil {
     HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
         "org.apache.hadoop.hive.ql.security.DummyAuthenticator");
     CliSessionState ss = new CliSessionState(conf);
-    assert ss != null;
     ss.in = System.in;
 
     SessionState oldSs = SessionState.get();
-
     restartSessions(canReuseSession, ss, oldSs);
-
     closeSession(oldSs);
 
     SessionState.start(ss);
@@ -1097,28 +1044,6 @@ public class QTestUtil {
     }
   }
 
-  protected void runCreateTableCmd(String createTableCmd) throws Exception {
-    int ecode = 0;
-    ecode = drv.run(createTableCmd).getResponseCode();
-    if (ecode != 0) {
-      throw new Exception("create table command: " + createTableCmd
-          + " failed with exit code= " + ecode);
-    }
-
-    return;
-  }
-
-  protected void runCmd(String cmd) throws Exception {
-    int ecode = 0;
-    ecode = drv.run(cmd).getResponseCode();
-    drv.close();
-    if (ecode != 0) {
-      throw new Exception("command: " + cmd
-          + " failed with exit code= " + ecode);
-    }
-    return;
-  }
-
   public void createSources() throws Exception {
     createSources(null);
   }
@@ -1234,7 +1159,6 @@ public class QTestUtil {
 
     CliSessionState ss = (CliSessionState) SessionState.get();
 
-
     String outFileExtension = getOutFileExtension(fileName);
     String stdoutName = null;
 
@@ -1257,7 +1181,7 @@ public class QTestUtil {
   }
 
   private void setSessionOutputs(String fileName, CliSessionState ss, File outf)
-      throws FileNotFoundException, Exception, UnsupportedEncodingException {
+      throws Exception {
     OutputStream fo = new BufferedOutputStream(new FileOutputStream(outf));
     if (ss.out != null) {
       ss.out.flush();
@@ -1304,8 +1228,8 @@ public class QTestUtil {
 
     String execEngine = conf.get("hive.execution.engine");
     conf.set("hive.execution.engine", "mr");
+
     CliSessionState ss = new CliSessionState(conf);
-    assert ss != null;
     ss.in = System.in;
     ss.out = System.out;
     ss.err = System.out;
@@ -1343,21 +1267,6 @@ public class QTestUtil {
     return cliDriver.processLine(q1);
   }
 
-  public int executeOne(String tname) {
-    String q = qMap.get(tname);
-
-    if (q.indexOf(";") == -1) {
-      return -1;
-    }
-
-    String q1 = q.substring(0, q.indexOf(";") + 1);
-    String qrest = q.substring(q.indexOf(";") + 1);
-    qMap.put(tname, qrest);
-
-    System.out.println("Executing " + q1);
-    return cliDriver.processLine(q1);
-  }
-
   public int execute(String tname) {
     return drv.run(qMap.get(tname)).getResponseCode();
   }
@@ -1514,20 +1423,6 @@ public class QTestUtil {
     return ".out";
   }
 
-  public void convertSequenceFileToTextFile() throws Exception {
-    // Create an instance of hive in order to create the tables
-    testWarehouse = conf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE);
-    db = Hive.get(conf);
-
-    // Move all data from dest4_sequencefile to dest4
-    drv
-      .run("FROM dest4_sequencefile INSERT OVERWRITE TABLE dest4 SELECT dest4_sequencefile.*");
-
-    // Drop dest4_sequencefile
-    db.dropTable(Warehouse.DEFAULT_DATABASE_NAME, "dest4_sequencefile",
-        true, true);
-  }
-
   public QTestProcessExecResult checkNegativeResults(String tname, Exception e) throws Exception {
 
     String outFileExtension = getOutFileExtension(tname);
@@ -1588,35 +1483,6 @@ public class QTestUtil {
     return result;
   }
 
-  public QTestProcessExecResult checkParseResults(String tname, ASTNode tree) throws Exception {
-
-    if (tree != null) {
-      String outFileExtension = getOutFileExtension(tname);
-
-      File parseDir = new File(outDir, "parse");
-      String expf = outPath(parseDir.toString(), tname.concat(outFileExtension));
-
-      File outf = null;
-      outf = new File(logDir);
-      outf = new File(outf, tname.concat(outFileExtension));
-
-      FileWriter outfd = new FileWriter(outf);
-      outfd.write(tree.toStringTree());
-      outfd.close();
-
-      QTestProcessExecResult exitVal = executeDiffCommand(outf.getPath(), expf, false, false);
-
-      if (overWrite) {
-        overwriteResults(outf.getPath(), expf);
-        return QTestProcessExecResult.createWithoutOutput(0);
-      }
-
-      return exitVal;
-    } else {
-      throw new Exception("Parse tree is null");
-    }
-  }
-
   /**
    * Given the current configurations (e.g., hadoop version and execution mode), return
    * the correct file name to compare with the current test run output.
@@ -1629,7 +1495,6 @@ public class QTestUtil {
     // List of configurations. Currently the list consists of hadoop version and execution mode only
     List<String> configs = new ArrayList<String>();
     configs.add(this.clusterType.toString());
-    configs.add(this.hadoopVer);
 
     Deque<String> stack = new LinkedList<String>();
     StringBuilder sb = new StringBuilder();
@@ -1675,7 +1540,6 @@ public class QTestUtil {
     return exitVal;
   }
 
-
   public QTestProcessExecResult checkCompareCliDriverResults(String tname, List<String> outputs)
       throws Exception {
     assert outputs.size() > 1;
@@ -1831,13 +1695,6 @@ public class QTestUtil {
     return pd.parse(qMap.get(tname));
   }
 
-  public void resetParser() throws SemanticException {
-    pd = new ParseDriver();
-    queryState = new QueryState.Builder().withHiveConf(conf).build();
-    sem = new SemanticAnalyzer(queryState);
-  }
-
-
   public List<Task<?>> analyzeAST(ASTNode ast) throws Exception {
 
     // Do semantic analysis and plan generation
@@ -1852,10 +1709,6 @@ public class QTestUtil {
     return sem.getRootTasks();
   }
 
-  public TreeMap<String, String> getQMap() {
-    return qMap;
-  }
-
   /**
    * QTestSetup defines test fixtures which are reused across testcases,
    * and are needed before any test can be run
@@ -1972,7 +1825,6 @@ public class QTestUtil {
             .withLogDir(logDir)
             .withClusterType(MiniClusterType.none)
             .withConfDir(null)
-            .withHadoopVer("0.20")
             .withInitScript(initScript == null ? defaultInitScript : initScript)
             .withCleanupScript(cleanupScript == null ? defaultCleanupScript : cleanupScript)
             .withLlapIo(false)

http://git-wip-us.apache.org/repos/asf/hive/blob/1e048df3/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java
index 77de3fa..a7972ec 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java
@@ -16,12 +16,11 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hive.ql.parse;
+
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.Serializable;
-import java.util.List;
 
 import org.apache.hadoop.hive.cli.control.AbstractCliConfig;
 import org.apache.hadoop.hive.cli.control.CliAdapter;
@@ -30,7 +29,6 @@ import org.apache.hadoop.hive.ql.QTestArguments;
 import org.apache.hadoop.hive.ql.QTestProcessExecResult;
 import org.apache.hadoop.hive.ql.QTestUtil;
 import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
-import org.apache.hadoop.hive.ql.exec.Task;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -41,8 +39,9 @@ public class CoreParseNegative extends CliAdapter{
 
   private static QTestUtil qt;
 
-  static CliConfigs.ParseNegativeConfig cliConfig = new CliConfigs.ParseNegativeConfig();
-  static boolean firstRun;
+  private static CliConfigs.ParseNegativeConfig cliConfig = new CliConfigs.ParseNegativeConfig();
+  private static boolean firstRun;
+
   public CoreParseNegative(AbstractCliConfig testCliConfig) {
     super(testCliConfig);
   }
@@ -54,26 +53,26 @@ public class CoreParseNegative extends CliAdapter{
     String initScript = cliConfig.getInitScript();
     String cleanupScript = cliConfig.getCleanupScript();
     firstRun = true;
+
     try {
-      String hadoopVer = cliConfig.getHadoopVersion();
       qt = new QTestUtil(
           QTestArguments.QTestArgumentsBuilder.instance()
             .withOutDir(cliConfig.getResultsDir())
             .withLogDir(cliConfig.getLogDir())
             .withClusterType(miniMR)
             .withConfDir(null)
-            .withHadoopVer(hadoopVer)
             .withInitScript(initScript)
             .withCleanupScript(cleanupScript)
             .withLlapIo(false)
             .build());
 
       qt.newSession();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
       System.err.flush();
-      throw new RuntimeException("Unexpected exception in static initialization",e);
+      throw new RuntimeException("Unexpected exception in static initialization", e);
     }
   }
 
@@ -88,24 +87,24 @@ public class CoreParseNegative extends CliAdapter{
 
   @Override
   @AfterClass
-  public void shutdown() throws Exception {
+  public void shutdown() {
     String reason = "clear post test effects";
     try {
       qt.clearPostTestEffects();
       reason = "shutdown";
       qt.shutdown();
+
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
       System.err.flush();
-      throw new RuntimeException("Unexpected exception in " + reason,e);
+      throw new RuntimeException("Unexpected exception in " + reason, e);
     }
   }
 
-  static String debugHint = "\nSee ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, "
+  private static String debugHint = "\nSee ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, "
      + "or check ./ql/target/surefire-reports or ./itests/qtest/target/surefire-reports/ for specific test cases logs.";
 
-
   @Override
   public void runTest(String tname, String fname, String fpath) throws Exception {
     long startTime = System.currentTimeMillis();
@@ -121,7 +120,7 @@ public class CoreParseNegative extends CliAdapter{
       qt.cliInit(new File(fpath));
 
       ASTNode tree = qt.parseQuery(fname);
-      List<Task<? extends Serializable>> tasks = qt.analyzeAST(tree);
+      qt.analyzeAST(tree);
       fail("Unexpected success for query: " + fname + debugHint);
     }
     catch (ParseException pe) {